You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by "lancelly (via GitHub)" <gi...@apache.org> on 2024/04/23 07:20:10 UTC

[PR] Enhance estimatation of FI memory usage [iotdb]

lancelly opened a new pull request, #12393:
URL: https://github.com/apache/iotdb/pull/12393

   Pipelines can be organized into a tree structure. This PR is used to estimate the memory usage during the execution of a pipeline. Currently, it is assumed that in the most extreme cases, the child pipelines under a ConsumeAllChildren type pipeline will all run simultaneously. Therefore, when considering memory usage, all children must be accounted for. For ConsumeChildrenOneByOne type pipelines, only some of the child pipelines will run at the same time. Therefore, only the memory usage of these active children needs to be considered, along with the static memory occupied by the inactive 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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1592317226


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/MemoryMeasurable.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.queryengine.execution;
+
+public interface MemoryMeasurable {

Review Comment:
   Changed to Accountable.



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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2098197224

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2095186924

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "JackieTien97 (via GitHub)" <gi...@apache.org>.
JackieTien97 merged PR #12393:
URL: https://github.com/apache/iotdb/pull/12393


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1590580947


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

Review Comment:
   Done.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanContext.java:
##########
@@ -91,6 +98,10 @@ public class LocalExecutionPlanContext {
   // use AtomicReference not for thread-safe, just for updating same field in different pipeline
   private AtomicReference<List<Long>> timePartitions = new AtomicReference<>();
 
+  /** Records the parent of each pipeline. The order of each list does not matter for now. */
+  private Map<PlanNodeId, List<PipelineMemoryEstimator>> parentPlanNodeIdToMemoryEstimator =

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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1590580899


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

Review Comment:
   Removed.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1590580783


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

Review Comment:
   done



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "JackieTien97 (via GitHub)" <gi...@apache.org>.
JackieTien97 commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1577089745


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java:
##########
@@ -209,6 +209,8 @@ public enum PlanNodeType {
   EXPLAIN_ANALYZE((short) 90),
 
   PIPE_OPERATE_SCHEMA_QUEUE_REFERENCE((short) 91),
+
+  FULL_OUTER_TIME_JOIN_REFERENCE((short) 92),

Review Comment:
   what's this for?



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LocalExecutionPlanContext.java:
##########
@@ -91,6 +98,10 @@ public class LocalExecutionPlanContext {
   // use AtomicReference not for thread-safe, just for updating same field in different pipeline
   private AtomicReference<List<Long>> timePartitions = new AtomicReference<>();
 
+  /** Records the parent of each pipeline. The order of each list does not matter for now. */
+  private Map<PlanNodeId, List<PipelineMemoryEstimator>> parentPlanNodeIdToMemoryEstimator =

Review Comment:
   remember to set this map to null, after you check the memory, this Map may occupy huge memory when there are many devices;



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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2095272672

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "JackieTien97 (via GitHub)" <gi...@apache.org>.
JackieTien97 commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1590950784


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/LastQueryOperator.java:
##########
@@ -173,4 +178,15 @@ public long calculateRetainedSizeAfterCallingNext() {
     }
     return max;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {

Review Comment:
   You should think about the memory size of `tsBlockBuilder` field.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/UpdateLastCacheOperator.java:
##########
@@ -89,4 +94,14 @@ protected void appendLastValueToTsBlockBuilder(long lastTime, TsPrimitiveType la
     LastQueryUtil.appendLastValue(
         tsBlockBuilder, lastTime, fullPath.getFullPath(), lastValue.getStringValue(), dataType);
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {
+    return INSTANCE_SIZE

Review Comment:
   where did you add the class size of `MeasurementPath` since you only add `EstimatedSizeOfPartialPathWithoutClassSize` here.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java:
##########
@@ -2947,6 +2949,18 @@ public List<Operator> dealWithConsumeAllChildrenPipelineBreaker(
       for (PlanNode localChild : node.getChildren()) {
         Operator childOperation = localChild.accept(this, context);
         parentPipelineChildren.add(childOperation);
+        // if we don't create extra pipeline, the root of the child pipeline should be current root
+        // for example, we have IdentitySinkNode -> DeviceViewNode -> [ScanNode, ScanNode, ScanNode]
+        // the parent of the pipeline of ScanNode should be IdentitySinkNode in the map, otherwise
+        // we will lose the information of these pipelines
+        List<PipelineMemoryEstimator> childPipelineMemoryEstimators =
+            context.getParentPlanNodeIdToMemoryEstimator().get(localChild.getPlanNodeId());
+        if (childPipelineMemoryEstimators != null) {
+          context.getParentPlanNodeIdToMemoryEstimator().remove(localChild.getPlanNodeId());
+          context
+              .getParentPlanNodeIdToMemoryEstimator()
+              .put(node.getPlanNodeId(), childPipelineMemoryEstimators);
+        }

Review Comment:
   What does this mean, and the case is incorrect, because `DeviceViewNode` won't appear in `dealWithConsumeAllChildrenPipelineBreaker`.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/LastCacheScanOperator.java:
##########
@@ -83,4 +87,11 @@ public long calculateRetainedSizeAfterCallingNext() {
   public PlanNodeId getSourceId() {
     return sourceId;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {
+    return INSTANCE_SIZE

Review Comment:
   also think about the memory usage of `tsBlock` field.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/LastQueryTransformOperator.java:
##########
@@ -117,4 +122,13 @@ public void close() throws Exception {
     }
     tsBlockBuilder = null;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {

Review Comment:
   also think about `tsBlockBuilder` field's memory size.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/MemoryMeasurable.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.queryengine.execution;
+
+public interface MemoryMeasurable {

Review Comment:
   what's the difference with `Accountable` interface.



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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2095107647

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1592316379


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/LastQueryOperator.java:
##########
@@ -173,4 +178,15 @@ public long calculateRetainedSizeAfterCallingNext() {
     }
     return max;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {

Review Comment:
   Done. I added the size of tsBlockBuilder in all operators that have this field.



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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2095302936

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimatation of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2071616292

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimatation of FI memory usage [iotdb]

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2071684287

   ## [Codecov](https://app.codecov.io/gh/apache/iotdb/pull/12393?dropdown=coverage&src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: Patch coverage is `96.51163%` with `3 lines` in your changes are missing coverage. Please review.
   > Project coverage is 43.19%. Comparing base [(`f4804a4`)](https://app.codecov.io/gh/apache/iotdb/commit/f4804a4144076397e254208a150d2762ce716f31?dropdown=coverage&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) to head [(`1b63668`)](https://app.codecov.io/gh/apache/iotdb/pull/12393?dropdown=coverage&src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   > Report is 44 commits behind head on master.
   
   | [Files](https://app.codecov.io/gh/apache/iotdb/pull/12393?dropdown=coverage&src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [...ueryengine/plan/planner/LocalExecutionPlanner.java](https://app.codecov.io/gh/apache/iotdb/pull/12393?src=pr&el=tree&filepath=iotdb-core%2Fdatanode%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fiotdb%2Fdb%2Fqueryengine%2Fplan%2Fplanner%2FLocalExecutionPlanner.java&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-aW90ZGItY29yZS9kYXRhbm9kZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaW90ZGIvZGIvcXVlcnllbmdpbmUvcGxhbi9wbGFubmVyL0xvY2FsRXhlY3V0aW9uUGxhbm5lci5qYXZh) | 57.14% | [3 Missing :warning: ](https://app.codecov.io/gh/apache/iotdb/pull/12393?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #12393      +/-   ##
   ============================================
   - Coverage     46.40%   43.19%   -3.22%     
     Complexity       69       69              
   ============================================
     Files          3378     3088     -290     
     Lines        207497   190358   -17139     
     Branches      24948    22672    -2276     
   ============================================
   - Hits          96291    82221   -14070     
   + Misses       111206   108137    -3069     
   ```
   
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/iotdb/pull/12393?dropdown=coverage&src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#issuecomment-2095261780

   ## [![Quality Gate Failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-failed-20px.png 'Quality Gate Failed')](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393) **Quality Gate failed**  
   Failed conditions  
   ![](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/failed-16px.png '') [0.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_iotdb&pullRequest=12393&metric=new_coverage&view=list) (required ≥ 80%)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_iotdb&pullRequest=12393)
   
   


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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1592314017


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java:
##########
@@ -2947,6 +2949,18 @@ public List<Operator> dealWithConsumeAllChildrenPipelineBreaker(
       for (PlanNode localChild : node.getChildren()) {
         Operator childOperation = localChild.accept(this, context);
         parentPipelineChildren.add(childOperation);
+        // if we don't create extra pipeline, the root of the child pipeline should be current root
+        // for example, we have IdentitySinkNode -> DeviceViewNode -> [ScanNode, ScanNode, ScanNode]
+        // the parent of the pipeline of ScanNode should be IdentitySinkNode in the map, otherwise
+        // we will lose the information of these pipelines
+        List<PipelineMemoryEstimator> childPipelineMemoryEstimators =
+            context.getParentPlanNodeIdToMemoryEstimator().get(localChild.getPlanNodeId());
+        if (childPipelineMemoryEstimators != null) {
+          context.getParentPlanNodeIdToMemoryEstimator().remove(localChild.getPlanNodeId());
+          context
+              .getParentPlanNodeIdToMemoryEstimator()
+              .put(node.getPlanNodeId(), childPipelineMemoryEstimators);
+        }

Review Comment:
   I updated the comment.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/LastCacheScanOperator.java:
##########
@@ -83,4 +87,11 @@ public long calculateRetainedSizeAfterCallingNext() {
   public PlanNodeId getSourceId() {
     return sourceId;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {
+    return INSTANCE_SIZE

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


Re: [PR] Enhance estimate of FI memory usage [iotdb]

Posted by "lancelly (via GitHub)" <gi...@apache.org>.
lancelly commented on code in PR #12393:
URL: https://github.com/apache/iotdb/pull/12393#discussion_r1592315583


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/UpdateLastCacheOperator.java:
##########
@@ -89,4 +94,14 @@ protected void appendLastValueToTsBlockBuilder(long lastTime, TsPrimitiveType la
     LastQueryUtil.appendLastValue(
         tsBlockBuilder, lastTime, fullPath.getFullPath(), lastValue.getStringValue(), dataType);
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {
+    return INSTANCE_SIZE

Review Comment:
   Now I refactored EstimatedSizeOfPartialPathWithoutClassSize to EstimatedSizeOfPartialPath in which the INSTANCE_SIZE of PartialPath is calculated. I removed INSTANCE_SIZE of PartialPath in other classes.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/last/LastQueryTransformOperator.java:
##########
@@ -117,4 +122,13 @@ public void close() throws Exception {
     }
     tsBlockBuilder = null;
   }
+
+  @Override
+  public long getEstimatedMemoryUsageInBytes() {

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