You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2019/12/26 07:58:52 UTC

[GitHub] [hadoop-ozone] timmylicheng opened a new pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

timmylicheng opened a new pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393
 
 
   ## What changes were proposed in this pull request?
   #HDDS-1590 Aggregate bytesWrite metrics from container to datanode and all the way to SCM.
   (Please fill in changes proposed in this fix)
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-1590
   
   (Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HDDS-XXXX. Fix a typo in YYY.)
   
   Please replace this section with the link to the Apache JIRA)
   
   ## How was this patch tested?
   Tested in env:
   # TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_2f0b4887_a6a4_4ab5_ba41_36aa5cfc9056 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_2f0b4887_a6a4_4ab5_ba41_36aa5cfc9056{hostname="VM_50_210_centos"} 102400
   
   # TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_9b3b4890_b347_4e14_9b84_53d499474473 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_9b3b4890_b347_4e14_9b84_53d499474473{hostname="VM_50_210_centos"} 307200
   
   # TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_dee39c86_ec69_4edf_bd05_3c5eb16a6ca9 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_dee39c86_ec69_4edf_bd05_3c5eb16a6ca9{hostname="VM_50_210_centos"} 204800
   
   # TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_398226ba_298d_4b72_9f66_cda4046b037b counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_398226ba_298d_4b72_9f66_cda4046b037b{hostname="VM_50_210_centos"} 11468800
   
   # TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_09c23825_6e8b_4a9d_9d61_603f360c3300 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_09c23825_6e8b_4a9d_9d61_603f360c3300{hostname="VM_50_210_centos"} 5
   
   (Please explain how this patch was tested. Ex: unit tests, manual tests)
   (If this patch involves UI changes, please attach a screen-shot; otherwise, remove this)
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-588299820
 
 
   > > Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.
   > 
   > Thanks to explain it.
   > 
   > I understand that it's good to calculate the number. I don't understand why is it added to the `StorageContainerDatanodeProtocol.proto` and sent back to the SCM. It seems to be a metric which can be fetched directly from the datanode. We don't need to send it to the SCM.
   > 
   > Unless the SCM needs this data for any **decision** (to create new pipelines, replica containers, etc..). I might missed it but as I see this is not the case (but please let me now if you have such a plan...).
   
   Yea I think you already have the picture. We plan to expose pipeline heavy load in terms of bytes written to make pipeline lifecycle more balanced. 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382498294
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -159,21 +159,14 @@ protected void initializePipelineState() throws IOException {
     }
   }
 
-  @Override
-  public synchronized Pipeline createPipeline(ReplicationType type,
-      ReplicationFactor factor) throws IOException {
-    lock.writeLock().lock();
-    try {
-      Pipeline pipeline = pipelineFactory.create(type, factor);
-      pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
-          pipeline.getProtobufMessage().toByteArray());
-      stateManager.addPipeline(pipeline);
-      nodeManager.addPipeline(pipeline);
-      metrics.incNumPipelineAllocated();
+  private void recordMetricsForPipeline(Pipeline pipeline) {
+    switch (pipeline.getType()) {
+    case STAND_ALONE:
       if (pipeline.isOpen()) {
 
 Review comment:
   is this isOpen check duplicated? 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng closed pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng closed pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393
 
 
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi merged pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi merged pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393
 
 
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382687388
 
 

 ##########
 File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
 ##########
 @@ -579,15 +581,33 @@ public boolean isExist(HddsProtos.PipelineID pipelineId) {
         RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineId).getId()));
   }
 
+  private long calculateBytesWriteForRelatedPipeline(
 
 Review comment:
   Updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] elek commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
elek commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-588123110
 
 
   > Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.
   
   Thanks to explain it.
   
   I understand that it's good to calculate the number. I don't understand why is it added to the `StorageContainerDatanodeProtocol.proto` and sent back to the SCM. It seems to be a metric which can be fetched directly from the datanode. We don't need to send it to the SCM.
   
   Unless the SCM needs this data for any **decision** (to create new pipelines, replica containers, etc..). I might missed it but as I see this is not the case (but please let me now if you have such a plan...).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-587351678
 
 
   > > Thanks to post this patch @timmylicheng. Overall it looks good, but I am not sure about the motivation.
   > > Why is it required to send metrics from one service to an other one and report from there? The metrics can be published by the datanode itself and can be summarized / organized by the metrics collector.
   > > Is this information required for any of the decision made by the SCM?
   > > I am pretty sure that I missed the context of this patch. Can you please help me to understand the goals?
   > 
   > Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.
   
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r362137731
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -137,11 +150,16 @@ void incNumPipelineCreated() {
   }
 
   /**
-   * Get the number of pipeline created.
-   * @return number of pipeline
+   * Increments the number of bytes by delta that write into the pipeline.
    */
-  long getNumPipelineCreated() {
-    return numPipelineCreated.value();
+  void incNumPipelineBytesWrite(Pipeline pipeline, long delta) {
 
 Review comment:
   I have updated the logic 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r383124312
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -159,21 +159,14 @@ protected void initializePipelineState() throws IOException {
     }
   }
 
-  @Override
-  public synchronized Pipeline createPipeline(ReplicationType type,
-      ReplicationFactor factor) throws IOException {
-    lock.writeLock().lock();
-    try {
-      Pipeline pipeline = pipelineFactory.create(type, factor);
-      pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
-          pipeline.getProtobufMessage().toByteArray());
-      stateManager.addPipeline(pipeline);
-      nodeManager.addPipeline(pipeline);
-      metrics.incNumPipelineAllocated();
+  private void recordMetricsForPipeline(Pipeline pipeline) {
+    switch (pipeline.getType()) {
+    case STAND_ALONE:
       if (pipeline.isOpen()) {
 
 Review comment:
   Updated

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-569851216
 
 
   > There is no unit test found. Please add corresponding unit test.
   
   In terms of unit test, this change is aggregating bytesWritten stats from containerData to Ratis server and to SCM thru pipeline report. AFAIK, there is no available test framework who goes all the way. I tried MiniOzoneCluster and it didn't contain containerData in its Ratis server. Need to spend more time investigating it.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-590239263
 
 
   +1.  Will commit it shortly. 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r361925666
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -56,11 +56,13 @@
   private @Metric MutableCounterLong numPipelineReportProcessingFailed;
   private @Metric MutableCounterLong numPipelineContainSameDatanodes;
   private Map<PipelineID, MutableCounterLong> numBlocksAllocated;
+  private Map<PipelineID, MutableCounterLong> numPipelineBytesWrite;
 
 Review comment:
   Change name to numBytesWritten.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382691375
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -137,11 +150,12 @@ void incNumPipelineCreated() {
   }
 
   /**
-   * Get the number of pipeline created.
-   * @return number of pipeline
+   * Increments the number of bytes by delta that write into the pipeline.
 
 Review comment:
   Update to total

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-569035004
 
 
   The acceptance details link doesn't work. Wasn't able to see what errors happen. Don't expect this change to affect acceptance test tho.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382687517
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
 ##########
 @@ -104,6 +106,12 @@ protected void processPipelineReport(PipelineReport report,
 
     setReportedDatanode(pipeline, dn);
     setPipelineLeaderId(report, pipeline, dn);
+    // ONE replica pipeline doesn't have leader flag
+    if (report.getIsLeader() ||
+        pipeline.getFactor() == HddsProtos.ReplicationFactor.ONE) {
+      pipeline.setLeaderId(dn.getUuid());
+      metrics.incNumPipelineBytesWrite(pipeline, report.getBytesWrite());
 
 Review comment:
   Updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382687332
 
 

 ##########
 File path: hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
 ##########
 @@ -240,6 +240,7 @@ message ContainerAction {
 message PipelineReport {
   required PipelineID pipelineID = 1;
   required bool isLeader = 2;
+  optional uint64 bytesWrite = 3;
 
 Review comment:
   Updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-585594255
 
 
   > Thanks to post this patch @timmylicheng. Overall it looks good, but I am not sure about the motivation.
   > Why is it required to send metrics from one service to an other one and report from there? The metrics can be published by the datanode itself and can be summarized / organized by the metrics collector.
   > 
   > Is this information required for any of the decision made by the SCM?
   > 
   > I am pretty sure that I missed the context of this patch. Can you please help me to understand the goals?
   
   Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382495084
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -102,15 +106,24 @@ void createPerPipelineMetrics(Pipeline pipeline) {
     numBlocksAllocated.put(pipeline.getId(), new MutableCounterLong(Interns
         .info(getBlockAllocationMetricName(pipeline),
             "Number of blocks allocated in pipeline " + pipeline.getId()), 0L));
+    numBytesWritten.put(pipeline.getId(), new MutableCounterLong(Interns
+        .info(getBytesWriteMetricName(pipeline),
+            "Number of bytes written into pipeline " + pipeline.getId()), 0L));
   }
 
   public static String getBlockAllocationMetricName(Pipeline pipeline) {
     return "NumBlocksAllocated-" + pipeline.getType() + "-" + pipeline
         .getFactor() + "-" + pipeline.getId().getId();
   }
 
+  public static String getBytesWriteMetricName(Pipeline pipeline) {
 
 Review comment:
   2

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382492535
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
 ##########
 @@ -104,6 +106,12 @@ protected void processPipelineReport(PipelineReport report,
 
     setReportedDatanode(pipeline, dn);
     setPipelineLeaderId(report, pipeline, dn);
+    // ONE replica pipeline doesn't have leader flag
+    if (report.getIsLeader() ||
+        pipeline.getFactor() == HddsProtos.ReplicationFactor.ONE) {
+      pipeline.setLeaderId(dn.getUuid());
+      metrics.incNumPipelineBytesWrite(pipeline, report.getBytesWrite());
 
 Review comment:
   same as above

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382492270
 
 

 ##########
 File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
 ##########
 @@ -579,15 +581,33 @@ public boolean isExist(HddsProtos.PipelineID pipelineId) {
         RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineId).getId()));
   }
 
+  private long calculateBytesWriteForRelatedPipeline(
 
 Review comment:
   BytesWritten?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r362137886
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -56,11 +56,13 @@
   private @Metric MutableCounterLong numPipelineReportProcessingFailed;
   private @Metric MutableCounterLong numPipelineContainSameDatanodes;
   private Map<PipelineID, MutableCounterLong> numBlocksAllocated;
+  private Map<PipelineID, MutableCounterLong> numPipelineBytesWrite;
 
 Review comment:
   Updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-588329463
 
 
   > There is no unit test found. Please add corresponding unit test.
   
   @ChenSammi Unit test is added. Please take a look.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r383117685
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -159,21 +159,14 @@ protected void initializePipelineState() throws IOException {
     }
   }
 
-  @Override
-  public synchronized Pipeline createPipeline(ReplicationType type,
-      ReplicationFactor factor) throws IOException {
-    lock.writeLock().lock();
-    try {
-      Pipeline pipeline = pipelineFactory.create(type, factor);
-      pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
-          pipeline.getProtobufMessage().toByteArray());
-      stateManager.addPipeline(pipeline);
-      nodeManager.addPipeline(pipeline);
-      metrics.incNumPipelineAllocated();
+  private void recordMetricsForPipeline(Pipeline pipeline) {
+    switch (pipeline.getType()) {
+    case STAND_ALONE:
       if (pipeline.isOpen()) {
 
 Review comment:
   In this caseļ¼Œ metrics.createPerPipelineMetrics(pipeline)  is executed twice for standalone pipeline.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382687579
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -102,15 +106,24 @@ void createPerPipelineMetrics(Pipeline pipeline) {
     numBlocksAllocated.put(pipeline.getId(), new MutableCounterLong(Interns
         .info(getBlockAllocationMetricName(pipeline),
             "Number of blocks allocated in pipeline " + pipeline.getId()), 0L));
+    numBytesWritten.put(pipeline.getId(), new MutableCounterLong(Interns
+        .info(getBytesWriteMetricName(pipeline),
+            "Number of bytes written into pipeline " + pipeline.getId()), 0L));
   }
 
   public static String getBlockAllocationMetricName(Pipeline pipeline) {
     return "NumBlocksAllocated-" + pipeline.getType() + "-" + pipeline
         .getFactor() + "-" + pipeline.getId().getId();
   }
 
+  public static String getBytesWriteMetricName(Pipeline pipeline) {
 
 Review comment:
   Updated.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng edited a comment on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng edited a comment on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-588299820
 
 
   > > Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.
   > 
   > Thanks to explain it.
   > 
   > I understand that it's good to calculate the number. I don't understand why is it added to the `StorageContainerDatanodeProtocol.proto` and sent back to the SCM. It seems to be a metric which can be fetched directly from the datanode. We don't need to send it to the SCM.
   > 
   > Unless the SCM needs this data for any **decision** (to create new pipelines, replica containers, etc..). I might missed it but as I see this is not the case (but please let me now if you have such a plan...).
   
   Yea I think you already have the picture. We plan to expose pipeline heavy load in terms of bytes written to make pipeline lifecycle more balanced. @elek 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng opened a new pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng opened a new pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393
 
 
   ## What changes were proposed in this pull request?
   #HDDS-1590 Aggregate bytesWrite metrics from container to datanode and all the way to SCM.
   (Please fill in changes proposed in this fix)
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-1590
   
   (Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HDDS-XXXX. Fix a typo in YYY.)
   
   Please replace this section with the link to the Apache JIRA)
   
   ## How was this patch tested?
   Tested in env:
   TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_2f0b4887_a6a4_4ab5_ba41_36aa5cfc9056 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_2f0b4887_a6a4_4ab5_ba41_36aa5cfc9056{hostname="VM_50_210_centos"} 102400
   
   TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_9b3b4890_b347_4e14_9b84_53d499474473 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_9b3b4890_b347_4e14_9b84_53d499474473{hostname="VM_50_210_centos"} 307200
   
   TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_dee39c86_ec69_4edf_bd05_3c5eb16a6ca9 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_dee39c86_ec69_4edf_bd05_3c5eb16a6ca9{hostname="VM_50_210_centos"} 204800
   
   TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_398226ba_298d_4b72_9f66_cda4046b037b counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_398226ba_298d_4b72_9f66_cda4046b037b{hostname="VM_50_210_centos"} 11468800
   
   TYPE scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_09c23825_6e8b_4a9d_9d61_603f360c3300 counter
   scm_pipeline_metrics_num_pipeline_bytes_write_ratis_three_09c23825_6e8b_4a9d_9d61_603f360c3300{hostname="VM_50_210_centos"} 5
   
   (Please explain how this patch was tested. Ex: unit tests, manual tests)
   (If this patch involves UI changes, please attach a screen-shot; otherwise, remove this)
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382492060
 
 

 ##########
 File path: hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
 ##########
 @@ -240,6 +240,7 @@ message ContainerAction {
 message PipelineReport {
   required PipelineID pipelineID = 1;
   required bool isLeader = 2;
+  optional uint64 bytesWrite = 3;
 
 Review comment:
   bytesWritten?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] elek commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
elek commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-584052314
 
 
   Thanks to post this patch @timmylicheng. Overall it looks good, but I am not sure about the motivation.
   Why is it required to send metrics from one service to an other one and report from there? The metrics can be published by the datanode itself and can be summarized / organized by the metrics collector.
   
   Is this information required for any of the decision made by the SCM?
   
   I am pretty sure that I missed the context of this patch. Can you please help me to understand the goals?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382691320
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -159,21 +159,14 @@ protected void initializePipelineState() throws IOException {
     }
   }
 
-  @Override
-  public synchronized Pipeline createPipeline(ReplicationType type,
-      ReplicationFactor factor) throws IOException {
-    lock.writeLock().lock();
-    try {
-      Pipeline pipeline = pipelineFactory.create(type, factor);
-      pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
-          pipeline.getProtobufMessage().toByteArray());
-      stateManager.addPipeline(pipeline);
-      nodeManager.addPipeline(pipeline);
-      metrics.incNumPipelineAllocated();
+  private void recordMetricsForPipeline(Pipeline pipeline) {
+    switch (pipeline.getType()) {
+    case STAND_ALONE:
       if (pipeline.isOpen()) {
 
 Review comment:
   Works as a kind of protection. Current way to create STAND_ALONE pipeline is to put the state as OPEN indeed. But it's not guaranteed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-587351914
 
 
   > > > Thanks to post this patch @timmylicheng. Overall it looks good, but I am not sure about the motivation.
   > > > Why is it required to send metrics from one service to an other one and report from there? The metrics can be published by the datanode itself and can be summarized / organized by the metrics collector.
   > > > Is this information required for any of the decision made by the SCM?
   > > > I am pretty sure that I missed the context of this patch. Can you please help me to understand the goals?
   > > 
   > > 
   > > Sure. This patch is to summarize the bytes in for every pipeline, which requires the sum of each container that is allocated to the pipeline. The bytes in stats for each container shall be collected on datanode to be near real-time. So that I collect them up by datanode reports and aggregate by pipeline on SCM.
   
   @elek Wonder if you have further questions. :)

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r382495617
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -137,11 +150,12 @@ void incNumPipelineCreated() {
   }
 
   /**
-   * Get the number of pipeline created.
-   * @return number of pipeline
+   * Increments the number of bytes by delta that write into the pipeline.
 
 Review comment:
   delta data or total data?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#discussion_r361925835
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java
 ##########
 @@ -137,11 +150,16 @@ void incNumPipelineCreated() {
   }
 
   /**
-   * Get the number of pipeline created.
-   * @return number of pipeline
+   * Increments the number of bytes by delta that write into the pipeline.
    */
-  long getNumPipelineCreated() {
-    return numPipelineCreated.value();
+  void incNumPipelineBytesWrite(Pipeline pipeline, long delta) {
 
 Review comment:
   The reported BytesWritten is delta or sum up? 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM

Posted by GitBox <gi...@apache.org>.
timmylicheng commented on issue #393: HDDS-1590 Aggregate bytes write metrics from container to datanode and to SCM
URL: https://github.com/apache/hadoop-ozone/pull/393#issuecomment-590084154
 
 
   /retest

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org