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 2021/01/13 16:49:01 UTC

[GitHub] [calcite] devozerov opened a new pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

devozerov opened a new pull request #2325:
URL: https://github.com/apache/calcite/pull/2325


   


----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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


   Merged via https://github.com/apache/calcite/commit/2ee4d


----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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


   Adding slow tests as this touches planner.


----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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


   Merged via https://github.com/apache/calcite/commit/2ee4d


----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;

Review comment:
       The custom trait doesn't have an `equals` method and relies on Java reference comparison, which is sufficient for the proper behavior. 
   For the avoidance of doubts, I added equals/hashCode to the trait class and also changed `satisfies` to use `equals` instead of the reference equality check. Please see https://github.com/apache/calcite/pull/2325/commits/915d5ff30fdc97e3afb58101619dc6385821be4e




----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;

Review comment:
       The custom trait doesn't have an `equals` method and relies on Java reference comparison, which is sufficient for the proper behavior. 
   To avoid any doubts, I added `equals/hashCode` to the trait class and also changed `satisfies` to use `equals` instead of the reference equality check. Please see https://github.com/apache/calcite/pull/2325/commits/915d5ff30fdc97e3afb58101619dc6385821be4e




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       O I see. So the goal was to remove that Sort? 
   
   It won't hurt though to verify that the final two Rels has the trait set: 
   
   ```
   NONE,[0,1],FROM
   NONE,[0,1],TO
   ```




----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       The `plus` method replaces the existing trait definitions internally. 
   
   Before: `NONE,[0,1],FROM`
   After:  `NONE,[0,1],TO`




----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       Or maybe the proposal is to check traits before and after the `plus` operation?




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       O I see. So the goal was to remove that Sort? 
   
   It won't hurt though to verify that the final two Rels has the trait set separately:
   
   ```
   NONE,[0,1],FROM
   NONE,[0,1],TO
   ```




----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       The goal was to ensure that only enforces that are really required are applied. Before the fix, the original node already has the collation that satisfies the output traits, yet, the `Sort` operator was added to MEMO.
   
   I slightly improved the test to address your concerns: https://github.com/apache/calcite/pull/2325/commits/0aedf2e3dc2a59f1f3366acd374e349fbdb01ea4




----------------------------------------------------------------
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.

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



[GitHub] [calcite] devozerov commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       This is not the goal of the test. Rels have two different traits - a collation and a custom one. But the collation of the original node satisfies the collation of the target node. Before the fix, Calcite added an unnecessary Sort on top of the original node. After the fix, Calcite enforces only the custom trait.




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       I see. In this case, can you also check in your unit test that for those two Rels, only the FROM and TO traits are different (others have skipped the conversion)?




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia closed pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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


   


----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       I see. In this case, can you also check in your unit test that for those two Rels, only the FROM and TO traits are different (others have skipped the conversion).




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;

Review comment:
       Will this be `this.equals(trait)`?




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia closed pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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


   


----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;

Review comment:
       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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       O I see. So the goal was to remove that Sort? 
   
   It won't hurt though to verify 
   
   ```
   Before: NONE,[0,1],FROM
   After: NONE,[0,1],TO
   ```




----------------------------------------------------------------
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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2325: [CALCITE-4466] Do not invoke RelTraitDef.convert when the source trait satisfies the target trait

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



##########
File path: core/src/test/java/org/apache/calcite/plan/volcano/MultipleTraitConversionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.ImmutableIntList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests that ensures that we do not add enforcers for the already satisfied traits.
+ * See https://issues.apache.org/jira/browse/CALCITE-4466 for more information.
+ */
+public class MultipleTraitConversionTest {
+  @SuppressWarnings("ConstantConditions")
+  @Test public void testMultipleTraitConversion() {
+    VolcanoPlanner planner = new VolcanoPlanner();
+
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+    planner.addRelTraitDef(CustomTraitDef.INSTANCE);
+    planner.setNoneConventionHasInfiniteCost(false);
+
+    RelOptCluster cluster = newCluster(planner);
+
+    RelTraitSet fromTraits = cluster.traitSetOf(RelCollations.of(ImmutableIntList.of(0, 1)));
+
+    RelTraitSet toTraits = fromTraits
+        .plus(RelCollations.of(0))
+        .plus(CustomTrait.TO);
+
+    CustomLeafRel rel = new CustomLeafRel(cluster, fromTraits);
+    planner.setRoot(rel);
+    planner.changeTraitsUsingConverters(rel, toTraits);
+
+    // Make sure that the equivalence set contains only the original and converted rels.
+    // It should not contain the collation enforcer, because the "from" collation already
+    // satisfies the "to" collation.
+    List<RelNode> rels = planner.getSubset(rel).set.rels;
+    assertEquals(2, rels.size());
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomLeafRel));
+    assertTrue(rels.stream().anyMatch(r -> r instanceof CustomTraitEnforcer));
+  }
+
+  /**
+   * Leaf rel.
+   */
+  private static class CustomLeafRel extends PlannerTests.TestLeafRel {
+    CustomLeafRel(RelOptCluster cluster, RelTraitSet traits) {
+      super(cluster, traits, CustomLeafRel.class.getSimpleName());
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomLeafRel(getCluster(), traitSet);
+    }
+
+    @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+        RelMetadataQuery mq) {
+      return planner.getCostFactory().makeTinyCost();
+    }
+  }
+
+  /**
+   * An enforcer used by the custom trait def.
+   */
+  private static class CustomTraitEnforcer extends SingleRel {
+    private CustomTraitEnforcer(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+      super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+      return new CustomTraitEnforcer(getCluster(), traitSet, inputs.get(0));
+    }
+  }
+
+  /**
+   * Custom trait.
+   */
+  private static class CustomTrait implements RelTrait {
+
+    private static final CustomTrait FROM = new CustomTrait("FROM");
+    private static final CustomTrait TO = new CustomTrait("TO");
+
+    private final String label;
+
+    private CustomTrait(String label) {
+      this.label = label;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override public RelTraitDef getTraitDef() {
+      return CustomTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+      return this == trait;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+      // No-op
+    }
+
+    @Override public String toString() {
+      return label;
+    }
+  }
+
+  /**
+   * Custom trait definition.
+   */
+  private static class CustomTraitDef extends RelTraitDef<CustomTrait> {
+
+    private static final CustomTraitDef INSTANCE = new CustomTraitDef();
+
+    @Override public Class<CustomTrait> getTraitClass() {
+      return CustomTrait.class;
+    }
+
+    @Override public String getSimpleName() {
+      return "custom";
+    }
+
+    @Override public @Nullable RelNode convert(
+        RelOptPlanner planner,
+        RelNode rel,
+        CustomTrait toTrait,
+        boolean allowInfiniteCostConverters
+    ) {
+      return new CustomTraitEnforcer(
+          rel.getCluster(),
+          rel.getTraitSet().plus(toTrait),

Review comment:
       I might not understand correctly:
   
   should here be `remove existing then plus`?




----------------------------------------------------------------
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.

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