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 2020/05/08 22:45:05 UTC

[GitHub] [calcite] xndai commented on a change in pull request #1953: [CALCITE-3896] Top down trait request

xndai commented on a change in pull request #1953:
URL: https://github.com/apache/calcite/pull/1953#discussion_r422397445



##########
File path: core/src/main/java/org/apache/calcite/plan/PhysicalNode.java
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Physical node in a planner that is capable of doing
+ * physical trait propagation and derivation.
+ */
+public interface PhysicalNode extends RelNode {
+
+  /**
+   * Pass required traitset from parent node to child nodes,
+   * returns new node after traits is passed down.
+   */
+  default RelNode passThrough(RelTraitSet required) {
+    Pair<RelTraitSet, List<RelTraitSet>> p = passThroughTraits(required);
+    if (p == null) {
+      return null;
+    }
+    int size = getInputs().size();
+    assert size == p.right.size();
+    List<RelNode> list = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      RelNode n = RelOptRule.convert(getInput(i), p.right.get(i));
+      list.add(n);
+    }
+    return copy(p.left, list);
+  }
+
+  /**
+   * Pass required traitset from parent node to child nodes,
+   * returns a pair of traits after traits is passed down.
+   *
+   * <p>Pair.left: the new traitset
+   * <p>Pair.right: the list of required traitsets for child nodes
+   */
+  default Pair<RelTraitSet, List<RelTraitSet>> passThroughTraits(
+      RelTraitSet required) {
+    throw new RuntimeException("Not implemented!");
+  }
+
+  /**
+   * Derive traitset from child node, returns new node after
+   * traits derivation.
+   */
+  default RelNode derive(RelTraitSet childTraits, int childId) {
+    Pair<RelTraitSet, List<RelTraitSet>> p = deriveTraits(childTraits, childId);
+    if (p == null) {
+      return null;
+    }
+    int size = getInputs().size();
+    assert size == p.right.size();
+    List<RelNode> list = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      RelNode node = getInput(i);
+      node = RelOptRule.convert(node, p.right.get(i));
+      list.add(node);
+    }
+    return copy(p.left, list);
+  }
+
+  /**
+   * Derive traitset from child node, returns a pair of traits after
+   * traits derivation.
+   *
+   * <p>Pair.left: the new traitset
+   * <p>Pair.right: the list of required traitsets for child nodes
+   */
+  default Pair<RelTraitSet, List<RelTraitSet>> deriveTraits(
+      RelTraitSet childTraits, int childId) {
+    throw new RuntimeException("Not implemented!");
+  }
+
+  /**
+   * Given a list of child traitsets,
+   * inputTraits.size() == getInput().size(),
+   * returns node list after traits derivation. This method is called
+   * ONLY when the derive mode is OMAKASE.
+   */
+  default List<RelNode> derive(List<List<RelTraitSet>> inputTraits) {
+    throw new RuntimeException("Not implemented!");
+  }
+
+  /**
+   * Returns mode of derivation.
+   */
+  default DeriveMode getDeriveMode() {
+    return DeriveMode.LEFT_FIRST;

Review comment:
       I am thinking maybe we don't want to provide a default mode. Each individual physical node would have to explicitly specify derive mode. So it's more clear.

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
##########
@@ -146,6 +165,32 @@ public RelSubset getSubset(RelTraitSet traits) {
     return null;
   }
 
+  public int getSeedSize() {
+    if (seeds.isEmpty()) {
+      seeds.addAll(rels);
+    }
+    return seeds.size();
+  }
+
+  public boolean hasNextPhysicalNode() {
+    while (relCursor < rels.size()) {
+      RelNode node = rels.get(relCursor);
+      if (node instanceof PhysicalNode
+          && node.getConvention() != Convention.NONE) {

Review comment:
       The 2nd condition should have been an assert IMO.

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableConvention.java
##########
@@ -46,6 +49,22 @@ public String getName() {
     return "ENUMERABLE";
   }
 
+  @Override public RelNode enforce(
+      final RelNode input,
+      final RelTraitSet required) {
+    RelNode rel = input;
+    if (input.getConvention() != INSTANCE) {
+      rel = ConventionTraitDef.INSTANCE.convert(
+          input.getCluster().getPlanner(),
+          input, INSTANCE, true);
+    }
+    RelCollation collation = required.getTrait(RelCollationTraitDef.INSTANCE);
+    if (collation != null && !collation.getFieldCollations().isEmpty()) {

Review comment:
       When will collation be not null but filed collations are empty?

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/OptimizeTask.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.Convention;
+import org.apache.calcite.plan.DeriveMode;
+import org.apache.calcite.plan.PhysicalNode;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import org.apiguardian.api.API;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <code>OptimizeTask</code> represents the optimization task
+ * of VolcanoPlanner.
+ */
+@API(since = "1.23", status = API.Status.INTERNAL)
+abstract class OptimizeTask {
+
+  static final Logger LOGGER = CalciteTrace.getPlannerTaskTracer();
+
+  static OptimizeTask create(RelNode node) {
+    if (node instanceof RelSubset) {
+      return new OptSubsetTask((RelSubset) node);
+    }
+    return new OptRelNodeTask(node);
+  }
+
+  final VolcanoPlanner planner;
+  final int id;
+
+  OptimizeTask(RelNode node) {
+    planner = (VolcanoPlanner) node.getCluster().getPlanner();
+    id = planner.nextTaskId++;
+    LOGGER.debug("Scheduled task(id={}) for {}", id, node);
+  }
+
+  abstract boolean hasSubTask();
+
+  abstract OptimizeTask nextSubTask();
+
+  abstract void execute();
+
+  /**
+   * Task State
+   */
+  public enum State {
+    SCHEDULED,
+    EXECUTING,
+    COMPLETED
+  }
+
+  /**
+   * Task for optimizing RelNode.
+   */
+  static class OptRelNodeTask extends OptimizeTask {
+    final RelNode node;
+    int nextId = 0; // next child index
+
+    OptRelNodeTask(RelNode node) {
+      super(node);
+      this.node = node;
+    }
+
+    @Override boolean hasSubTask() {
+      int size = node.getInputs().size();
+      while (nextId < size) {
+        RelSubset subset = (RelSubset) node.getInput(nextId);
+        if (subset.taskState == null) {
+          // not yet scheduled
+          return true;
+        } else {
+          // maybe a cycle if it is not completed
+          nextId++;
+        }
+      }
+
+      return false;
+    }
+
+    @Override OptimizeTask nextSubTask() {
+      RelNode child = node.getInput(nextId++);
+      return new OptSubsetTask((RelSubset) child);
+    }
+
+    @Override void execute() {
+      if (!(node instanceof PhysicalNode)
+          || ((PhysicalNode) node).getDeriveMode() == DeriveMode.PROHIBITED
+          || !planner.isSeedNode(node)) {
+        LOGGER.debug("Completed task(id={}) for {}", id, node);
+        return;
+      }
+
+      PhysicalNode rel = (PhysicalNode) node;
+      DeriveMode mode = rel.getDeriveMode();
+      int arity = node.getInputs().size();
+      // for OMAKASE
+      List<List<RelTraitSet>> inputTraits = new ArrayList<>(arity);
+
+      for (int i = 0; i < arity; i++) {
+        int childId = i;
+        if (mode == DeriveMode.RIGHT_FIRST) {
+          childId = arity - i - 1;
+        }
+
+        RelSubset input = (RelSubset) node.getInput(childId);
+        RelTraitSet required = input.getTraitSet();
+        List<RelTraitSet> traits = new ArrayList<>();
+        inputTraits.add(traits);
+
+        final int numSubset = input.set.subsets.size();
+        for (int j = 0; j < numSubset; j++) {
+          RelSubset subset = input.set.subsets.get(j);
+          if (!subset.isDelivered()
+              // TODO: should use matching type to determine
+              || required.equals(subset.getTraitSet())) {
+            continue;
+          }
+
+          if (mode == DeriveMode.OMAKASE) {
+            traits.add(subset.getTraitSet());
+          } else {
+            RelNode newRel = rel.derive(subset.getTraitSet(), childId);
+            if (newRel != null) {
+              RelSubset relSubset = planner.register(newRel, node);
+              assert relSubset.set == planner.getSubset(node).set;
+            }
+          }
+        }
+
+        if (mode == DeriveMode.LEFT_FIRST
+            || mode == DeriveMode.RIGHT_FIRST) {
+          break;
+        }
+      }
+
+      if (mode == DeriveMode.OMAKASE) {
+        List<RelNode> relList = rel.derive(inputTraits);
+        for (RelNode relNode : relList) {
+          planner.register(relNode, node);
+        }
+      }
+
+      LOGGER.debug("Completed task(id={}) for {}", id, node);
+    }
+
+    @Override public String toString() {
+      return "#" + id + ":OptRelNodeTask{ " + node + " }";
+    }
+  }
+
+  /**
+   * Task for optimizing RelSubset.
+   */
+  static class OptSubsetTask extends OptimizeTask {

Review comment:
       RelSubsetOptTask

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/OptimizeTask.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.Convention;
+import org.apache.calcite.plan.DeriveMode;
+import org.apache.calcite.plan.PhysicalNode;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import org.apiguardian.api.API;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <code>OptimizeTask</code> represents the optimization task
+ * of VolcanoPlanner.
+ */
+@API(since = "1.23", status = API.Status.INTERNAL)
+abstract class OptimizeTask {
+
+  static final Logger LOGGER = CalciteTrace.getPlannerTaskTracer();
+
+  static OptimizeTask create(RelNode node) {
+    if (node instanceof RelSubset) {
+      return new OptSubsetTask((RelSubset) node);
+    }
+    return new OptRelNodeTask(node);
+  }
+
+  final VolcanoPlanner planner;
+  final int id;
+
+  OptimizeTask(RelNode node) {
+    planner = (VolcanoPlanner) node.getCluster().getPlanner();
+    id = planner.nextTaskId++;
+    LOGGER.debug("Scheduled task(id={}) for {}", id, node);
+  }
+
+  abstract boolean hasSubTask();
+
+  abstract OptimizeTask nextSubTask();
+
+  abstract void execute();
+
+  /**
+   * Task State
+   */
+  public enum State {
+    SCHEDULED,
+    EXECUTING,
+    COMPLETED
+  }
+
+  /**
+   * Task for optimizing RelNode.
+   */
+  static class OptRelNodeTask extends OptimizeTask {
+    final RelNode node;
+    int nextId = 0; // next child index
+
+    OptRelNodeTask(RelNode node) {
+      super(node);
+      this.node = node;
+    }
+
+    @Override boolean hasSubTask() {
+      int size = node.getInputs().size();
+      while (nextId < size) {
+        RelSubset subset = (RelSubset) node.getInput(nextId);
+        if (subset.taskState == null) {
+          // not yet scheduled
+          return true;
+        } else {
+          // maybe a cycle if it is not completed
+          nextId++;
+        }
+      }
+
+      return false;
+    }
+
+    @Override OptimizeTask nextSubTask() {
+      RelNode child = node.getInput(nextId++);
+      return new OptSubsetTask((RelSubset) child);
+    }
+
+    @Override void execute() {
+      if (!(node instanceof PhysicalNode)
+          || ((PhysicalNode) node).getDeriveMode() == DeriveMode.PROHIBITED
+          || !planner.isSeedNode(node)) {
+        LOGGER.debug("Completed task(id={}) for {}", id, node);
+        return;
+      }
+
+      PhysicalNode rel = (PhysicalNode) node;
+      DeriveMode mode = rel.getDeriveMode();
+      int arity = node.getInputs().size();
+      // for OMAKASE
+      List<List<RelTraitSet>> inputTraits = new ArrayList<>(arity);
+
+      for (int i = 0; i < arity; i++) {
+        int childId = i;
+        if (mode == DeriveMode.RIGHT_FIRST) {
+          childId = arity - i - 1;
+        }
+
+        RelSubset input = (RelSubset) node.getInput(childId);
+        RelTraitSet required = input.getTraitSet();
+        List<RelTraitSet> traits = new ArrayList<>();
+        inputTraits.add(traits);
+
+        final int numSubset = input.set.subsets.size();
+        for (int j = 0; j < numSubset; j++) {
+          RelSubset subset = input.set.subsets.get(j);
+          if (!subset.isDelivered()
+              // TODO: should use matching type to determine
+              || required.equals(subset.getTraitSet())) {
+            continue;
+          }
+
+          if (mode == DeriveMode.OMAKASE) {
+            traits.add(subset.getTraitSet());
+          } else {
+            RelNode newRel = rel.derive(subset.getTraitSet(), childId);
+            if (newRel != null) {
+              RelSubset relSubset = planner.register(newRel, node);
+              assert relSubset.set == planner.getSubset(node).set;
+            }
+          }
+        }
+
+        if (mode == DeriveMode.LEFT_FIRST
+            || mode == DeriveMode.RIGHT_FIRST) {
+          break;
+        }
+      }
+
+      if (mode == DeriveMode.OMAKASE) {
+        List<RelNode> relList = rel.derive(inputTraits);
+        for (RelNode relNode : relList) {
+          planner.register(relNode, node);
+        }
+      }
+
+      LOGGER.debug("Completed task(id={}) for {}", id, node);
+    }
+
+    @Override public String toString() {
+      return "#" + id + ":OptRelNodeTask{ " + node + " }";
+    }
+  }
+
+  /**
+   * Task for optimizing RelSubset.
+   */
+  static class OptSubsetTask extends OptimizeTask {
+    final RelSubset subset;
+    final Map<RelTraitSet, RelSubset> derivedSubsets = new HashMap<>();
+
+    OptSubsetTask(RelSubset subset) {
+      super(subset);
+      this.subset = subset;
+      subset.taskState = State.SCHEDULED;
+      propagateTraits();

Review comment:
       any particular reason to do it in constructor?

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/OptimizeTask.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.Convention;
+import org.apache.calcite.plan.DeriveMode;
+import org.apache.calcite.plan.PhysicalNode;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import org.apiguardian.api.API;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <code>OptimizeTask</code> represents the optimization task
+ * of VolcanoPlanner.
+ */
+@API(since = "1.23", status = API.Status.INTERNAL)
+abstract class OptimizeTask {
+
+  static final Logger LOGGER = CalciteTrace.getPlannerTaskTracer();
+
+  static OptimizeTask create(RelNode node) {
+    if (node instanceof RelSubset) {
+      return new OptSubsetTask((RelSubset) node);
+    }
+    return new OptRelNodeTask(node);
+  }
+
+  final VolcanoPlanner planner;
+  final int id;
+
+  OptimizeTask(RelNode node) {
+    planner = (VolcanoPlanner) node.getCluster().getPlanner();
+    id = planner.nextTaskId++;
+    LOGGER.debug("Scheduled task(id={}) for {}", id, node);
+  }
+
+  abstract boolean hasSubTask();
+
+  abstract OptimizeTask nextSubTask();
+
+  abstract void execute();
+
+  /**
+   * Task State
+   */
+  public enum State {
+    SCHEDULED,
+    EXECUTING,
+    COMPLETED
+  }
+
+  /**
+   * Task for optimizing RelNode.
+   */
+  static class OptRelNodeTask extends OptimizeTask {
+    final RelNode node;
+    int nextId = 0; // next child index
+
+    OptRelNodeTask(RelNode node) {
+      super(node);
+      this.node = node;
+    }
+
+    @Override boolean hasSubTask() {
+      int size = node.getInputs().size();
+      while (nextId < size) {
+        RelSubset subset = (RelSubset) node.getInput(nextId);
+        if (subset.taskState == null) {
+          // not yet scheduled
+          return true;
+        } else {
+          // maybe a cycle if it is not completed
+          nextId++;
+        }
+      }
+
+      return false;
+    }
+
+    @Override OptimizeTask nextSubTask() {
+      RelNode child = node.getInput(nextId++);
+      return new OptSubsetTask((RelSubset) child);
+    }
+
+    @Override void execute() {
+      if (!(node instanceof PhysicalNode)
+          || ((PhysicalNode) node).getDeriveMode() == DeriveMode.PROHIBITED
+          || !planner.isSeedNode(node)) {
+        LOGGER.debug("Completed task(id={}) for {}", id, node);
+        return;
+      }
+
+      PhysicalNode rel = (PhysicalNode) node;
+      DeriveMode mode = rel.getDeriveMode();
+      int arity = node.getInputs().size();
+      // for OMAKASE
+      List<List<RelTraitSet>> inputTraits = new ArrayList<>(arity);
+
+      for (int i = 0; i < arity; i++) {
+        int childId = i;
+        if (mode == DeriveMode.RIGHT_FIRST) {
+          childId = arity - i - 1;
+        }
+
+        RelSubset input = (RelSubset) node.getInput(childId);
+        RelTraitSet required = input.getTraitSet();
+        List<RelTraitSet> traits = new ArrayList<>();
+        inputTraits.add(traits);
+
+        final int numSubset = input.set.subsets.size();
+        for (int j = 0; j < numSubset; j++) {
+          RelSubset subset = input.set.subsets.get(j);
+          if (!subset.isDelivered()
+              // TODO: should use matching type to determine
+              || required.equals(subset.getTraitSet())) {
+            continue;
+          }
+
+          if (mode == DeriveMode.OMAKASE) {
+            traits.add(subset.getTraitSet());
+          } else {
+            RelNode newRel = rel.derive(subset.getTraitSet(), childId);
+            if (newRel != null) {
+              RelSubset relSubset = planner.register(newRel, node);
+              assert relSubset.set == planner.getSubset(node).set;
+            }
+          }
+        }
+
+        if (mode == DeriveMode.LEFT_FIRST
+            || mode == DeriveMode.RIGHT_FIRST) {
+          break;
+        }
+      }
+
+      if (mode == DeriveMode.OMAKASE) {
+        List<RelNode> relList = rel.derive(inputTraits);
+        for (RelNode relNode : relList) {
+          planner.register(relNode, node);
+        }
+      }
+
+      LOGGER.debug("Completed task(id={}) for {}", id, node);
+    }
+
+    @Override public String toString() {
+      return "#" + id + ":OptRelNodeTask{ " + node + " }";
+    }
+  }
+
+  /**
+   * Task for optimizing RelSubset.

Review comment:
       Please briefly describe what it really does for optimizing RelSubset. For example - passing down the trait requirements of current RelSubset and add converters to the new derived subsets.

##########
File path: core/src/test/resources/saffron.properties
##########
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+calcite.planner.topdown.opt=true

Review comment:
       So the shipping configuration is false by default (CalciteSystemProperties), while the UT is set to true? I believe our UT should run on default shipping configs. In such case, we would need new test suite (or test matrix) with this config set to true.

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
##########
@@ -72,6 +74,23 @@
   RelSet equivalentSet;
   RelNode rel;
 
+  /**
+   * The position indicator of rel node that is to be processed.
+   */
+  private int relCursor = 0;
+
+  /**
+   * The relnodes after applying logical rules and physical rules,
+   * before trait propagation and enforcement.
+   */
+  final Set<RelNode> seeds = new HashSet<>();
+
+  /**

Review comment:
       Maybe an array list would serve the purpose?




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