You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/03/20 14:32:16 UTC

[GitHub] [iotdb] xingtanzjr opened a new pull request #5291: Implement the basic functionalities for DistributionPlanner

xingtanzjr opened a new pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291


   ## Description
   This PR mainly contains these contents:
   * Try the visit pattern for PlanNode. 
   * Implement the basic functionalities of DistributionPlanner. 
   * Supply more methods for PlanNode.
   
   
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] JackieTien97 commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830626521



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/common/DataRegion.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iotdb.db.mpp.common;
+
+/**
+ * This class is used to represent the data partition info including the DataRegionId and physical
+ * node IP address
+ */
+// TODO: (xingtanzjr) This class should be substituted with the class defined in Consensus level
+public class DataRegion {
+  private Integer dataRegionId;

Review comment:
       `int` will be better, but it's fine to keep it as this way as it will be replaced with the class defined in Consensus level later

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeAllocator.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.iotdb.db.mpp.sql.planner.plan.node;
+
+public class PlanNodeAllocator {

Review comment:
       ```suggestion
   public class PlanNodeIdAllocator {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/execution/InstanceState.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.iotdb.db.mpp.execution;
+
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+
+public enum InstanceState {

Review comment:
       Aha, I've already add `FragmentInstanceState` in this package, you can safely delete this class.

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });

Review comment:
       ```suggestion
         Map<DataRegion, List<SeriesScanNode>> sourceGroup = sources.stream().collect(Collectors.groupingBy(SeriesScanNode::getDataRegion));
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
##########
@@ -19,14 +19,17 @@
 package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
 
 import org.apache.iotdb.db.metadata.path.PartialPath;
-import org.apache.iotdb.db.mpp.common.OrderBy;
+import org.apache.iotdb.db.mpp.common.DataRegion;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
+import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeAllocator;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanVisitor;
+import org.apache.iotdb.db.sql.statement.component.OrderBy;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import com.google.common.collect.ImmutableList;
 
+import javax.xml.crypto.Data;

Review comment:
       ```suggestion
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });
+
+      // Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
+      // and make the
+      // new TimeJoinNode as the child of current TimeJoinNode
+      sourceGroup.forEach(
+          (dataRegion, seriesScanNodes) -> {
+            if (seriesScanNodes.size() == 1) {
+              root.addChild(seriesScanNodes.get(0));
+            } else {
+              // We clone a TimeJoinNode from root to make the params to be consistent
+              TimeJoinNode parentOfGroup = (TimeJoinNode) root.clone();
+              seriesScanNodes.forEach(parentOfGroup::addChild);
+              root.addChild(parentOfGroup);
+            }
+          });

Review comment:
       What if all `seriesScanNodes.size()` > 1, we will have a duplicated TimeJoinNode. We may simply choose the first entry of `sourceGroup` to generate the root `TimeJoinNode` and then add others as its children.




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830637079



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/common/DataRegion.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iotdb.db.mpp.common;
+
+/**
+ * This class is used to represent the data partition info including the DataRegionId and physical
+ * node IP address
+ */
+// TODO: (xingtanzjr) This class should be substituted with the class defined in Consensus level
+public class DataRegion {
+  private Integer dataRegionId;

Review comment:
       ok




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#issuecomment-1073434346


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=BUG) [23 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=SECURITY_HOTSPOT) [2 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=CODE_SMELL) [414 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=5291&resolved=false&types=CODE_SMELL)
   
   [![9.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '9.5%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=5291&metric=new_coverage&view=list) [9.5% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=5291&metric=new_coverage&view=list)  
   [![5.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/10-16px.png '5.1%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=5291&metric=new_duplicated_lines_density&view=list) [5.1% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=5291&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] JackieTien97 merged pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
JackieTien97 merged pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291


   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830638163



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });
+
+      // Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
+      // and make the
+      // new TimeJoinNode as the child of current TimeJoinNode
+      sourceGroup.forEach(
+          (dataRegion, seriesScanNodes) -> {
+            if (seriesScanNodes.size() == 1) {
+              root.addChild(seriesScanNodes.get(0));
+            } else {
+              // We clone a TimeJoinNode from root to make the params to be consistent
+              TimeJoinNode parentOfGroup = (TimeJoinNode) root.clone();
+              seriesScanNodes.forEach(parentOfGroup::addChild);
+              root.addChild(parentOfGroup);
+            }
+          });

Review comment:
       For the scenario you mentioned, I think we should put the `duplicated` TimeJoinNode in current stage, although we should delete it if the `duplicated` one and his parent are in the same Fragment. we can make the optimization in following step.
   
   By the way, your comment reminds me another optimization here. That is, if we have only one element in sourceGroup, we will produce a duplicated TimeJoinNode...




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830638330



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });
+
+      // Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
+      // and make the
+      // new TimeJoinNode as the child of current TimeJoinNode
+      sourceGroup.forEach(
+          (dataRegion, seriesScanNodes) -> {
+            if (seriesScanNodes.size() == 1) {
+              root.addChild(seriesScanNodes.get(0));
+            } else {
+              // We clone a TimeJoinNode from root to make the params to be consistent
+              TimeJoinNode parentOfGroup = (TimeJoinNode) root.clone();
+              seriesScanNodes.forEach(parentOfGroup::addChild);
+              root.addChild(parentOfGroup);
+            }
+          });

Review comment:
       I added a TODO here to remind us the optimization. Because this code is not really completed and let's complete it in later PR

##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });

Review comment:
       Done




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830637224



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanNodeAllocator.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.iotdb.db.mpp.sql.planner.plan.node;
+
+public class PlanNodeAllocator {

Review comment:
       Fixed




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830636789



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });

Review comment:
       Great!




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830636704



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/DistributionPlanner.java
##########
@@ -29,7 +39,214 @@ public DistributionPlanner(Analysis analysis, LogicalQueryPlan logicalPlan) {
     this.logicalPlan = logicalPlan;
   }
 
+  public PlanNode rewriteSource() {
+    SourceRewriter rewriter = new SourceRewriter();
+    return rewriter.visit(logicalPlan.getRootNode(), new DistributionPlanContext());
+  }
+
+  public PlanNode addExchangeNode(PlanNode root) {
+    ExchangeNodeAdder adder = new ExchangeNodeAdder();
+    return adder.visit(root, new NodeGroupContext());
+  }
+
   public DistributedQueryPlan planFragments() {
     return null;
   }
+
+  private class SourceRewriter extends SimplePlanNodeRewriter<DistributionPlanContext> {
+
+    //TODO: (xingtanzjr) implement the method visitDeviceMergeNode()
+    public PlanNode visitDeviceMerge(TimeJoinNode node, DistributionPlanContext context) {
+      return null;
+    }
+
+    public PlanNode visitTimeJoin(TimeJoinNode node, DistributionPlanContext context) {
+      TimeJoinNode root = (TimeJoinNode) node.clone();
+
+      // Step 1: Get all source nodes. For the node which is not source, add it as the child of
+      // current TimeJoinNode
+      List<SeriesScanNode> sources = new ArrayList<>();
+      for (PlanNode child : node.getChildren()) {
+        if (child instanceof SeriesScanNode) {
+          // If the child is SeriesScanNode, we need to check whether this node should be seperated
+          // into several splits.
+          SeriesScanNode handle = (SeriesScanNode) child;
+          Set<DataRegion> dataDistribution =
+              analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
+          // If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
+          // SeriesScanNode.
+          for (DataRegion dataRegion : dataDistribution) {
+            SeriesScanNode split = (SeriesScanNode) handle.clone();
+            split.setDataRegion(dataRegion);
+            sources.add(split);
+          }
+        } else if (child instanceof SeriesAggregateScanNode) {
+          // TODO: (xingtanzjr) We should do the same thing for SeriesAggregateScanNode. Consider to
+          // make SeriesAggregateScanNode
+          // and SeriesScanNode to derived from the same parent Class because they have similar
+          // process logic in many scenarios
+        } else {
+          // In a general logical query plan, the children of TimeJoinNode should only be
+          // SeriesScanNode or SeriesAggregateScanNode
+          // So this branch should not be touched.
+          root.addChild(visit(child, context));
+        }
+      }
+
+      // Step 2: For the source nodes, group them by the DataRegion.
+      Map<DataRegion, List<SeriesScanNode>> sourceGroup = new HashMap<>();
+      sources.forEach(
+          source -> {
+            List<SeriesScanNode> group =
+                sourceGroup.containsKey(source.getDataRegion())
+                    ? sourceGroup.get(source.getDataRegion())
+                    : new ArrayList<>();
+            group.add(source);
+            sourceGroup.put(source.getDataRegion(), group);
+          });
+
+      // Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
+      // and make the
+      // new TimeJoinNode as the child of current TimeJoinNode
+      sourceGroup.forEach(
+          (dataRegion, seriesScanNodes) -> {
+            if (seriesScanNodes.size() == 1) {
+              root.addChild(seriesScanNodes.get(0));
+            } else {
+              // We clone a TimeJoinNode from root to make the params to be consistent
+              TimeJoinNode parentOfGroup = (TimeJoinNode) root.clone();
+              seriesScanNodes.forEach(parentOfGroup::addChild);
+              root.addChild(parentOfGroup);
+            }
+          });

Review comment:
       Yes, we can do some optimization here




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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



[GitHub] [iotdb] xingtanzjr commented on a change in pull request #5291: [IoTDB-2660] Implement the basic functionalities for DistributionPlanner

Posted by GitBox <gi...@apache.org>.
xingtanzjr commented on a change in pull request #5291:
URL: https://github.com/apache/iotdb/pull/5291#discussion_r830637277



##########
File path: server/src/main/java/org/apache/iotdb/db/mpp/execution/InstanceState.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.iotdb.db.mpp.execution;
+
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+
+public enum InstanceState {

Review comment:
       Yes, this issue is led by the confusing auto-merge process




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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