You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2022/03/08 08:19:47 UTC

[GitHub] [calcite] wojustme opened a new pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

wojustme opened a new pull request #2740:
URL: https://github.com/apache/calcite/pull/2740


   Define a rule to pull up constants project under Sort
   Related JIRA: https://issues.apache.org/jira/browse/CALCITE-5035
   
   Just like:
   ```
   ===origin rel tree===
   LogicalSort(sort0=[$2], sort1=[$0], dir0=[ASC], dir1=[ASC])
     LogicalProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$7])
       LogicalFilter(condition=[=($7, 123)])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
   ===optimized rel tree===
   LogicalProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[123])
   |-LogicalSort(sort0=[$0], dir0=[ASC])
     |-LogicalProject(EMPNO=[$0], ENAME=[$1], DEPTNO=[$7])
       |-LogicalFilter(condition=[=($7, 123)])
         |-LogicalTableScan(table=[[CATALOG, SALES, EMP]])
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando commented on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando commented on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1061643518


   The changes applied so far are good to me, there are some unaddressed changes though, can you take care of them and I will review it all together? Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando edited a comment on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando edited a comment on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1061643518


   The changes applied so far are good to me, there are some unaddressed changes though (basically the "unresolved messages", I have marked as "resolved" the others you have already applied), can you take care of them and I will review it all together? Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando commented on a change in pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando commented on a change in pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#discussion_r821458303



##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();

Review comment:
       What's the rationale for using a `NavigableMap` \ `TreeMap` over a `HashMap` here?
   
   Below you use random access and you never iterate, I think `HashMap` would be a better fit unless I missed something.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando commented on a change in pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando commented on a change in pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#discussion_r821452100



##########
File path: core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
##########
@@ -712,6 +712,10 @@ private CoreRules() {}
   public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
       SortProjectTransposeRule.Config.DEFAULT.toRule();
 
+  /** Rule that pulles up constant literal under an {@link Sort}. */
+  public static final SortProjectPullUpConstantsRule SORT_ANY_PULL_UP_CONSTANTS =

Review comment:
       You use two names here for the same logical entity, `SortProjectPullUpConstantsRule` and `SORT_ANY_PULL_UP_CONSTANTS`, which can generate confusion.
   
   What about `SortPullUpConstantsRule`/`SORT_PULL_UP_CONSTANTS`? Mentioning `project` made me wonder if the project was to be expected as sort's input, which is not the case (that's why you use `ANY` in the constant name), after looking at the rule I understood it referred to where the constants are "moved".
   
   What do you think?
   
   PS: both of the original names can work for me, as long as we pick one in all places, you can ignore my suggestion if you don't like the names

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */

Review comment:
       I think we can drop the comment, what happens it's clear already

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
##########
@@ -712,6 +712,10 @@ private CoreRules() {}
   public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
       SortProjectTransposeRule.Config.DEFAULT.toRule();
 
+  /** Rule that pulles up constant literal under an {@link Sort}. */

Review comment:
       Nitpick:
   ```suggestion
     /** Rule that pulls up constant literal under a {@link Sort}. */
   ```

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.

Review comment:
       ```suggestion
       // None of the sort field expressions is constant. Nothing to do.
   ```

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.

Review comment:
       ```suggestion
           // Neither offset nor fetch in current sort, drop it.
   ```

##########
File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
##########
@@ -5530,6 +5530,58 @@ private void checkSemiJoinRuleOnAntiJoin(RelOptRule rule) {
         .checkUnchanged();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5035">[CALCITE-5035]
+   * Pull up constant project under sort</a>. */
+  @Test void testSortProjectPullUpConstants1() {

Review comment:
       You have different test cases for a reason, try to highlight what is the specificity in the name of the test instead of using `1`, `2` etc.
   
   It will help debug test failures, but it will also make the life of other people easier to understand if what they are looking for is already handled (tests are documentation too, and their names are a big part of it).
   
   For instance, here the focus looks like it's the absence of fetch/limit, maybe we can rename it as `testSortProjectPullUpConstantsNoFetchLimit`?
   
   The same comment applies for the other tests.

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.
+        newInputRel = input;
+      } else {
+        // Some offset or fetch exist in current sort.
+        newInputRel = sort.copy(sort.getTraitSet(), input, RelCollations.EMPTY);
+      }
+    } else {
+      // All field collations are not all constants.
+      final List<RelFieldCollation> newFieldCollations = new ArrayList<>();
+      for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+        final int fieldIndex = fieldCollation.getFieldIndex();
+        if (map.containsKey(fieldIndex)) {
+          continue;
+        }
+        newFieldCollations.add(fieldCollation);
+      }
+      final RelCollation newRelCollation = RelCollations.of(newFieldCollations);
+      newInputRel = sort.copy(sort.getTraitSet(), input, newRelCollation);
+    }
+
+    final RelBuilder relBuilder = call.builder();
+    relBuilder.push(newInputRel);
+
+    // Create a projection back again.
+    List<Pair<RexNode, String>> projects = new ArrayList<>();
+    for (RelDataTypeField field : sort.getRowType().getFieldList()) {
+      final int index = field.getIndex();
+      final RexNode constantNode = map.get(index);
+      final RexNode newExpr;
+      if (constantNode == null) {
+        newExpr = rexBuilder.makeInputRef(newInputRel, index);
+      } else {
+        // Re-generate the constant expression in the project.
+        RelDataType originalType =
+            sort.getRowType().getFieldList().get(projects.size()).getType();
+        if (!originalType.equals(constantNode.getType())) {
+          newExpr = rexBuilder.makeCast(originalType, constantNode, true);
+        } else {
+          newExpr = constantNode;
+        }
+      }
+      projects.add(Pair.of(newExpr, field.getName()));
+    }
+    relBuilder.project(Pair.left(projects), Pair.right(projects)); // inverse
+
+    final RelNode newRelNode = relBuilder.build();
+    call.transformTo(newRelNode);

Review comment:
       ```suggestion
       call.transformTo(relBuilder.build());
   ```

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.
+        newInputRel = input;
+      } else {
+        // Some offset or fetch exist in current sort.
+        newInputRel = sort.copy(sort.getTraitSet(), input, RelCollations.EMPTY);
+      }
+    } else {
+      // All field collations are not all constants.
+      final List<RelFieldCollation> newFieldCollations = new ArrayList<>();
+      for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+        final int fieldIndex = fieldCollation.getFieldIndex();
+        if (map.containsKey(fieldIndex)) {
+          continue;
+        }
+        newFieldCollations.add(fieldCollation);
+      }
+      final RelCollation newRelCollation = RelCollations.of(newFieldCollations);
+      newInputRel = sort.copy(sort.getTraitSet(), input, newRelCollation);
+    }
+
+    final RelBuilder relBuilder = call.builder();
+    relBuilder.push(newInputRel);
+
+    // Create a projection back again.
+    List<Pair<RexNode, String>> projects = new ArrayList<>();
+    for (RelDataTypeField field : sort.getRowType().getFieldList()) {
+      final int index = field.getIndex();
+      final RexNode constantNode = map.get(index);
+      final RexNode newExpr;
+      if (constantNode == null) {
+        newExpr = rexBuilder.makeInputRef(newInputRel, index);
+      } else {
+        // Re-generate the constant expression in the project.
+        RelDataType originalType =
+            sort.getRowType().getFieldList().get(projects.size()).getType();
+        if (!originalType.equals(constantNode.getType())) {
+          newExpr = rexBuilder.makeCast(originalType, constantNode, true);
+        } else {
+          newExpr = constantNode;
+        }
+      }
+      projects.add(Pair.of(newExpr, field.getName()));
+    }
+    relBuilder.project(Pair.left(projects), Pair.right(projects)); // inverse

Review comment:
       Not sure what the comment is saying, if it's important can you elaborate it a bit more?

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.
+        newInputRel = input;
+      } else {
+        // Some offset or fetch exist in current sort.
+        newInputRel = sort.copy(sort.getTraitSet(), input, RelCollations.EMPTY);
+      }
+    } else {
+      // All field collations are not all constants.
+      final List<RelFieldCollation> newFieldCollations = new ArrayList<>();
+      for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+        final int fieldIndex = fieldCollation.getFieldIndex();
+        if (map.containsKey(fieldIndex)) {
+          continue;
+        }
+        newFieldCollations.add(fieldCollation);
+      }
+      final RelCollation newRelCollation = RelCollations.of(newFieldCollations);
+      newInputRel = sort.copy(sort.getTraitSet(), input, newRelCollation);
+    }
+
+    final RelBuilder relBuilder = call.builder();
+    relBuilder.push(newInputRel);
+
+    // Create a projection back again.
+    List<Pair<RexNode, String>> projects = new ArrayList<>();
+    for (RelDataTypeField field : sort.getRowType().getFieldList()) {
+      final int index = field.getIndex();
+      final RexNode constantNode = map.get(index);
+      final RexNode newExpr;
+      if (constantNode == null) {
+        newExpr = rexBuilder.makeInputRef(newInputRel, index);
+      } else {
+        // Re-generate the constant expression in the project.
+        RelDataType originalType =
+            sort.getRowType().getFieldList().get(projects.size()).getType();

Review comment:
       Extract `sort.getRowType().getFieldList()` into a variable, it's a long expression and you use it already two times here, it will improve readability IMO.

##########
File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
##########
@@ -5530,6 +5530,58 @@ private void checkSemiJoinRuleOnAntiJoin(RelOptRule rule) {
         .checkUnchanged();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5035">[CALCITE-5035]
+   * Pull up constant project under sort</a>. */
+  @Test void testSortProjectPullUpConstants1() {
+    // The constant's count is more than sort's field, and without any fetch or offset
+    final String sql = "select e.empno, e.ename, e.deptno from sales.emp e\n"
+        + "where e.deptno = 123\n"
+        + "order by e.deptno, e.empno";
+    sql(sql)
+        .withRule(CoreRules.SORT_ANY_PULL_UP_CONSTANTS)
+        .check();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5035">[CALCITE-5035]
+   * Pull up constant project under sort</a>. */
+  @Test void testSortProjectPullUpConstants2() {
+    // Sort with some fetch and offset

Review comment:
       You could drop the comment by improving the test name as suggested above.

##########
File path: core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
##########
@@ -5530,6 +5530,58 @@ private void checkSemiJoinRuleOnAntiJoin(RelOptRule rule) {
         .checkUnchanged();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5035">[CALCITE-5035]
+   * Pull up constant project under sort</a>. */
+  @Test void testSortProjectPullUpConstants1() {
+    // The constant's count is more than sort's field, and without any fetch or offset
+    final String sql = "select e.empno, e.ename, e.deptno from sales.emp e\n"

Review comment:
       Nitpick:
   ```suggestion
       final String sql = "select e.empno, e.ename, e.deptno\n"
           + "from sales.emp e\n"
   ```
   If you accept this change, please be consistent in the other tests.
   
   PS: I see that in the test, both styles are used, but what I suggest seems preferred for long lines like in your case, up to you :)

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();

Review comment:
       What's the rationale for using a `NavigableMap`\`TreeMap` over a `HashMap` here?
   
   Below you use random access and you never iterate, I think `HashMap` would be a better fit unless I missed something.

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {

Review comment:
       ```suggestion
       if (fieldCollationIndexes.isEmpty()) {
   ```

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.
+        newInputRel = input;
+      } else {
+        // Some offset or fetch exist in current sort.
+        newInputRel = sort.copy(sort.getTraitSet(), input, RelCollations.EMPTY);
+      }
+    } else {
+      // All field collations are not all constants.

Review comment:
       ```suggestion
         // Some field collations are not constants.
   ```

##########
File path: core/src/main/java/org/apache/calcite/rel/rules/SortProjectPullUpConstantsRule.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.RelOptPredicateList;
+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.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Planner rule that removes constant keys from an
+ * {@link Sort}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * <p>Since the transformed relational expression has to match the original
+ * relational expression, the constants are placed in a projection above the
+ * reduced sort. If those constants are not used, another rule will remove
+ * them from the project.
+ */
+@Value.Enclosing
+public class SortProjectPullUpConstantsRule
+    extends RelRule<SortProjectPullUpConstantsRule.Config>
+    implements TransformationRule {
+
+  /** Creates an SortProjectPullUpConstantsRule. */
+  protected SortProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelCollation collation = sort.getCollation();
+    final List<Integer> fieldCollationIndexes = collation.getFieldCollations()
+        .stream()
+        .map(RelFieldCollation::getFieldIndex)
+        .collect(Collectors.toList());
+    if (fieldCollationIndexes.size() <= 0) {
+      return;
+    }
+
+    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelOptPredicateList predicates =
+        mq.getPulledUpPredicates(input);
+    if (RelOptPredicateList.isEmpty(predicates)) {
+      return;
+    }
+
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int fieldCollationIndex : fieldCollationIndexes) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(input, fieldCollationIndex);
+      if (predicates.constantMap.containsKey(ref)) {
+        map.put(fieldCollationIndex, predicates.constantMap.get(ref));
+      }
+    }
+
+    // None of the sort field expressions are constant. Nothing to do.
+    if (map.isEmpty()) {
+      return;
+    }
+
+    // Input of new top constant project.
+    final RelNode newInputRel;
+    if (map.size() == fieldCollationIndexes.size()) {
+      // All field collations are constants.
+      if (sort.offset == null && sort.fetch == null) {
+        // No any offset or fetch in current sort.
+        // Remove current sort.
+        newInputRel = input;
+      } else {
+        // Some offset or fetch exist in current sort.
+        newInputRel = sort.copy(sort.getTraitSet(), input, RelCollations.EMPTY);
+      }
+    } else {
+      // All field collations are not all constants.
+      final List<RelFieldCollation> newFieldCollations = new ArrayList<>();
+      for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+        final int fieldIndex = fieldCollation.getFieldIndex();
+        if (map.containsKey(fieldIndex)) {
+          continue;
+        }
+        newFieldCollations.add(fieldCollation);
+      }
+      final RelCollation newRelCollation = RelCollations.of(newFieldCollations);
+      newInputRel = sort.copy(sort.getTraitSet(), input, newRelCollation);
+    }
+
+    final RelBuilder relBuilder = call.builder();
+    relBuilder.push(newInputRel);
+
+    // Create a projection back again.
+    List<Pair<RexNode, String>> projects = new ArrayList<>();
+    for (RelDataTypeField field : sort.getRowType().getFieldList()) {
+      final int index = field.getIndex();
+      final RexNode constantNode = map.get(index);
+      final RexNode newExpr;
+      if (constantNode == null) {
+        newExpr = rexBuilder.makeInputRef(newInputRel, index);
+      } else {
+        // Re-generate the constant expression in the project.
+        RelDataType originalType =
+            sort.getRowType().getFieldList().get(projects.size()).getType();
+        if (!originalType.equals(constantNode.getType())) {
+          newExpr = rexBuilder.makeCast(originalType, constantNode, true);
+        } else {
+          newExpr = constantNode;
+        }

Review comment:
       Having in the `if` clause a negated statement increases the cognitive complexity of the code, we can swap the `then/else` branches and remove the negation:
   
   ```suggestion
           if (originalType.equals(constantNode.getType())) {
             newExpr = constantNode;
           } else {
             newExpr = rexBuilder.makeCast(originalType, constantNode, true);
           }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando commented on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando commented on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1062713378


   Don't worry @wojustme, my fault as well for not realising we already had this functionality. 
   Let's close this PR, we can keep discussing in the Jira ticket.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando commented on a change in pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando commented on a change in pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#discussion_r821452100



##########
File path: core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
##########
@@ -712,6 +712,10 @@ private CoreRules() {}
   public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
       SortProjectTransposeRule.Config.DEFAULT.toRule();
 
+  /** Rule that pulles up constant literal under an {@link Sort}. */
+  public static final SortProjectPullUpConstantsRule SORT_ANY_PULL_UP_CONSTANTS =

Review comment:
       You use two names here for the same logical entity, `SortProjectPullUpConstantsRule` and `SORT_ANY_PULL_UP_CONSTANTS`, which can generate confusion.
   
   What about `SortPullUpConstantsRule`/`SORT_PULL_UP_CONSTANTS`? Mentioning `project` made me wonder if the project was to be expected as sort's input, which is not the case (that's why you use `ANY` in the constant name), after looking at the rule I understood it referred to where the constants are "moved".
   
   What do you think?
   
   PS: both of the original names can work for me, as long as we pick one in all places, you can ignore my suggestion if you don't like the names
   
   In case you change the rule name, don't forget to update the Jira title (and the commit message during the final rebase/squash).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] wojustme commented on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
wojustme commented on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1061671565


   @asolimando 
   I am sorry for my eagerness.
   I think it's ready now.
   Please take your time to review this PR.
   Thanks a lot.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] asolimando closed pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
asolimando closed pull request #2740:
URL: https://github.com/apache/calcite/pull/2740


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] wojustme commented on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
wojustme commented on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1061641515


   @asolimando 
   Thanks your careful review, and it is very helpful to me.
   PR has updated, please review again.
   Please ping me to rebase this PR, when you think that it's good for you.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] wojustme commented on pull request #2740: [CALCITE-5035] Define a rule of SortProjectPullUpConstantsRule to pull up constant's project under Sort

Posted by GitBox <gi...@apache.org>.
wojustme commented on pull request #2740:
URL: https://github.com/apache/calcite/pull/2740#issuecomment-1062593740


   Hi @asolimando 
   Thanks for your guidance.
   But, I found that `CoreRules#SORT_REMOVE_CONSTANT_KEYS` do the same work.
   Thank you again, and I am very sorry to bother you.
   Some records in JIRA: https://issues.apache.org/jira/browse/CALCITE-5035


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org