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 2020/12/01 05:24:25 UTC

[GitHub] [ozone] smengcl opened a new pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

smengcl opened a new pull request #1601:
URL: https://github.com/apache/ozone/pull/1601


   This is an improved version of the previous fix (https://github.com/apache/ozone/pull/1552) for [HDDS-4404](https://issues.apache.org/jira/browse/HDDS-4404).
   
   The approach is relatively straightforward: DN will only save the latest `containerReport`/`nodeReport`/`pipelineReport` in `StateContext` (instead of the previous behavior where DN keeps putting all reports back on communication error.)
   
   ## Testing
   
   - Existing tests in `TestHeartbeatEndpointTask` passed locally.
   - New UTs added.
   - Internal scale test w/ 1M containers passed: recon unresponsive for 48 hours, DN heap usage kept at the same level -- w/o patch the DNs OOM'ed after 24 hours (All 15 DNs max heap set to 32GB, `reports` queue kept growing).


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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731972182


   > Thanks for the review @runzhiwang . In a previous revision I had a strict check in `StateContext#addReport`:
   > [22846f4#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240](https://github.com/apache/ozone/commit/22846f48ce34f3fcefa1de2d3bd97c4771c26742#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240)
   > 
   > which ensures only `CommandStatusReports` and `IncrementalContainerReport` would ever be added to `reports` queue. Later this is removed since @avijayanhwx suggested we don't really need that.
   > 
   > The only two paths where reports would be queued are `addReports` and `StateContext#putBackReports`, where `HeartbeatEndpointTask#putBackReports` makes sure we only put back `CommandStatusReports` and `IncrementalContainerReport`.
   > 
   > Would you elaborate on the integration test plan you have on your mind?
   
   @smengcl OOM is a serious problem, so we need strong test. If somebody change the existed two path, or add the third path, and put ContainerReport in the queue, OOM will happen, so we need some restrain. The integration test maybe only can cover the existed two path, for example, you can decrease the time interval of ReportPublisher#call and increase the time interval of EndpointStateMachine.EndPointStates call(), so that ContainerReport generated many times but has not been send out, then we check the reports do not contain ContainerReport.


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



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


[GitHub] [ozone] runzhiwang closed pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang closed pull request #1601:
URL: https://github.com/apache/ozone/pull/1601


   


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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-736765247


   Thanks @runzhiwang @avijayanhwx @GlenGeng for the review and comments.
   
   CI still has unrelated failures (`TestOzoneManagerHAWithData#testOMRestart` and `acceptance.ozone-om-ha-s3`). I will retrigger one more time.


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



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


[GitHub] [ozone] smengcl edited a comment on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731960577


   > @smengcl Thanks the patch. It looks good to me overall. But I think we need more strong integration test to cover the case: StateContext#reports will not contain any ContainerReport, NodeReport, and DatanodeReport.
   
   Thanks for the review @runzhiwang . In a previous revision I had a strict check in `StateContext#addReport`:
   https://github.com/apache/ozone/pull/1601/commits/22846f48ce34f3fcefa1de2d3bd97c4771c26742#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240
   
   which ensures only `CommandStatusReports` and `IncrementalContainerReport` would ever be added to `reports` queue. Later this is removed since @avijayanhwx suggested we don't really need that.
   
   The only two paths where reports would be queued are `addReports` and `StateContext#putBackReports`, where `HeartbeatEndpointTask#putBackReports` makes sure we only put back `CommandStatusReports` and `IncrementalContainerReport`.
   
   Would you elaborate on the integration test plan you have on your mind?


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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528492391



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -583,4 +631,16 @@ public void addEndpoint(InetSocketAddress endpoint) {
       this.reports.put(endpoint, new LinkedList<>());
     }
   }
+
+  public GeneratedMessage getContainerReports() {

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.

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



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


[GitHub] [ozone] runzhiwang removed a comment on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang removed a comment on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731881872


   @smengcl Hi, I still see OOM with this patch.
   ![image](https://user-images.githubusercontent.com/51938049/99922346-16129800-2d6b-11eb-8b61-ee7229a6c5be.png)
   


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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r527137852



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +217,32 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // Check report type
+        if (reportType.equals(containerReportsProtoName)) {
+          containerReport = report;
+        } else if (reportType.equals(nodeReportProtoName)) {
+          nodeReport = report;
+        } else if (reportType.equals(pipelineReportsProtoName)) {
+          pipelineReport = report;
+        } else if (reportType.equals(commandStatusReportsProtoName) ||
+            reportType.equals(incrementalContainerReportProtoName)) {
+          // report type is CommandStatusReports or IncrementalContainerReport
+          synchronized (reports) {
+            reports.get(endpoint).add(report);

Review comment:
       Yes `else` would work here. I was pushing this as an assertion check here just for now to see if I will get any UT failures. Will remove later.




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



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


[GitHub] [ozone] runzhiwang edited a comment on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang edited a comment on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731956707


   @smengcl Thanks the patch. It looks good to me overall. But I think we need more strong test to cover the case: StateContext#reports will not contain any ContainerReport, NodeReport, and DatanodeReport.


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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528002974



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       No container/node/pipeline should be fine. We need `synchroized` on `reports` because we want Map `get` and `add` not to get messed in between.




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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-742741887


   `TestDecommissionAndMaintenance#testNodeWithOpenPipelineCanBeDecommissioned` passed locally for me. Retriggering CI.


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



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


[GitHub] [ozone] runzhiwang commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528493460



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       one thread write containerReport and another thread read containerReport, it's not thread safe, we can use `AtomicReference<GeneratedMessage> containerReports`




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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
GlenGeng commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r529256780



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -72,7 +101,13 @@
   private final AtomicLong stateExecutionCount;
   private final ConfigurationSource conf;
   private final Set<InetSocketAddress> endpoints;
-  private final Map<InetSocketAddress, List<GeneratedMessage>> reports;
+  // Only the latest full report of each type is kept
+  private final AtomicReference<GeneratedMessage> containerReports;

Review comment:
       You may make them to be `volatile`, which is simpler but thread safe as well,




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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-732696270


   Summary of major changes to the latest diff:
   
   - Renamed `reports` to `incrementalReportsQueue` to better reflect the functionality of the map.
   - Added `acceptedIncrementalReportTypeSet` for rigorous report type checking.
   - `StateContext#addReport` now explicitly throws, instead of hiding any possible NPEs. Rationale: As I think again, those reports are all internally generated rather than dependent on direct user input. It's better to catch any problem early (if any).
   - `StateContext#addReport` also strictly checks for report type now. This intends to force any future additions of report types to be explicitly allowed. Only 5 report types (names) are expected:
   ```
   hadoop.hdds.NodeReportProto
   hadoop.hdds.ContainerReportsProto
   hadoop.hdds.IncrementalContainerReportProto
   hadoop.hdds.CommandStatusReportsProto
   hadoop.hdds.PipelineReportsProto
   ```
   - `StateContext#putBackReports` now checks ALL reports in the list to be put back. I'd expect the list to be small. And since the logic is run on each DataNode, it shouldn't become a performance bottleneck. Also added debug logging just in case we need to diagnose it in the future.
   - Added new UT `testPutBackReports` and `testReportQueueWithAddReports` to check that `putBackReports` and `getReports` behaves.
   
   Let me know if you think I missed anything, or it might be a bit over engineered.


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



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


[GitHub] [ozone] runzhiwang closed pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang closed pull request #1601:
URL: https://github.com/apache/ozone/pull/1601


   


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



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


[GitHub] [ozone] smengcl edited a comment on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-732696270


   Summary of major changes to the latest diff:
   
   - Renamed `reports` to `incrementalReportsQueue` to better reflect the functionality of the map.
   - Added `acceptedIncrementalReportTypeSet` for rigorous report type checking.
   - `StateContext#addReport` now explicitly throws, instead of hiding any possible NPEs. Rationale: As I think again, those reports are all internally generated rather than dependent on direct user input. It's better to catch any problem early (if any).
   - `StateContext#addReport` also strictly checks for report type now. This intends to force any future additions of report types to be explicitly allowed. Only 5 report types (names) are expected:
   ```
   hadoop.hdds.NodeReportProto
   hadoop.hdds.ContainerReportsProto
   hadoop.hdds.IncrementalContainerReportProto
   hadoop.hdds.CommandStatusReportsProto
   hadoop.hdds.PipelineReportsProto
   ```
   - `StateContext#putBackReports` now checks ALL reports in the list to be put back. I'd expect the list to be small. And since the logic is run on each DataNode, it shouldn't become a performance bottleneck. Also added debug logging just in case we need to diagnose it in the future.
   - Added new UT `testPutBackReports` and `testReportQueueWithAddReports` to check that `putBackReports` and `getReports` behaves.
   
   Let me know if you think I missed anything, or it might be a bit over overkill.


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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-741941863


   `TestStorageContainerManager#testBlockDeletionTransactions` was failing for a legitimate reason.
   The small issue is addressed by simply returning (instead of throwing) when `null` is passed into `StateContext#addReport`.


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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731956707


   @smengcl Thanks the patch. It looks good to me overall. I think we need more strong test to cover the case: StateContext#reports will not contain any ContainerReport, NodeReport, and DatanodeReport.


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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-730605471


   UT `TestStateContext#testReportAPIs` is not happy with my change in `StateContext#addReport`:
   https://github.com/apache/ozone/blob/a4f7e32b438a1ac74e23f70be3b24aac9a61e00c/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestStateContext.java#L68
   
   Mocked object `GeneratedMessage report` is fed to `addReport()`, so `report.getDescriptorForType()` returns `null`. As a result `report.getDescriptorForType().getFullName()` throws NPE in the UT.


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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528002974



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       No, container/node/pipeline should be fine. We need `synchroized` on `reports` because we want Map `get` and `add` not to get messed in between.




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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r532832000



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

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.

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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-736338966


   Waiting CI pass.


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



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


[GitHub] [ozone] GlenGeng commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
GlenGeng commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r527471776



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       Do we need to synchronize the get/set on these containerReport/nodeReport/pipelineReport ?




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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-732697355


   > > > Thanks for the review @runzhiwang . In a previous revision I had a strict check in `StateContext#addReport`:
   > > > [22846f4#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240](https://github.com/apache/ozone/commit/22846f48ce34f3fcefa1de2d3bd97c4771c26742#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240)
   > > > which ensures only `CommandStatusReports` and `IncrementalContainerReport` would ever be added to `reports` queue. Later this is removed since @avijayanhwx suggested we don't really need that.
   > > > The only two paths where reports would be queued are `addReports` and `StateContext#putBackReports`, where `HeartbeatEndpointTask#putBackReports` makes sure we only put back `CommandStatusReports` and `IncrementalContainerReport`.
   > > > Would you elaborate on the integration test plan you have on your mind?
   > > 
   > > 
   > > @smengcl OOM is a serious problem, so we need strong test. If somebody change the existed two path, or add the third path, and put ContainerReport in the queue, OOM will happen, so we need some restrain. The integration test maybe only can cover the existed two path, for example, you can decrease the time interval of ReportPublisher#call and increase the time interval of EndpointStateMachine.EndPointStates call(), so that ContainerReport generated many times but has not been send out, then we check the reports do not contain ContainerReport.
   > 
   > Thanks @runzhiwang .
   > 
   > I think a better and easier way to check this behavior is to add a new UT that calls `addReports` and `putBackReports` repetitively, then check the `reports` queue length. Should only expect the queue grow when report type is `CommandStatusReport` or `IncrementalContainerReport`, but not any other ones. What do you think?
   
   @smengcl I agree.


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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-736119908


   Reopen pr to trigger CI


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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731881872


   @smengcl Hi, I still see OOM with this patch.
   ![image](https://user-images.githubusercontent.com/51938049/99922346-16129800-2d6b-11eb-8b61-ee7229a6c5be.png)
   


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



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


[GitHub] [ozone] runzhiwang closed pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang closed pull request #1601:
URL: https://github.com/apache/ozone/pull/1601


   


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



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


[GitHub] [ozone] runzhiwang commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528437351



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -583,4 +631,16 @@ public void addEndpoint(InetSocketAddress endpoint) {
       this.reports.put(endpoint, new LinkedList<>());
     }
   }
+
+  public GeneratedMessage getContainerReports() {

Review comment:
       @VisiableForTesting ?




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



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


[GitHub] [ozone] runzhiwang commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528493460



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       one thread write containerReport and another thread read containerReport, it's not thread safe, we can use `AtomicReference<GeneratedMessage> containerReport`




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



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


[GitHub] [ozone] smengcl commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731960577


   > @smengcl Thanks the patch. It looks good to me overall. But I think we need more strong integration test to cover the case: StateContext#reports will not contain any ContainerReport, NodeReport, and DatanodeReport.
   
   Thanks for the review @runzhiwang . In a previous revision I had a strict check in `StateContext#addReport`:
   https://github.com/apache/ozone/pull/1601/commits/22846f48ce34f3fcefa1de2d3bd97c4771c26742#diff-4e474a0cbb28091ed4fab8ee80878c3acf7e15711fba2a0d83bacd56ea8f5707L237-L240
   
   which ensures only `CommandStatusReports` and `IncrementalContainerReport` would ever be adding to `reports` queue. Later this is removed since @avijayanhwx suggested we don't really need that.
   
   The only two paths where reports would be queued are `addReports` and `StateContext#putBackReports`, where `HeartbeatEndpointTask#putBackReports` makes sure we only put back `CommandStatusReports` and `IncrementalContainerReport`.
   
   Would you elaborate on the integration test plan you have on your mind?


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



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


[GitHub] [ozone] runzhiwang commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528493460



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +211,29 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      // TODO: Check report.getDescriptorForType() != null as well?
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // We only keep the latest container, node and pipeline report
+        if (reportType.equals(CONTAINER_REPORTS_PROTO_NAME)) {

Review comment:
       one thread write containerReport and another thread read containerReport, it's not safe, we can use `AtomicReference<GeneratedMessage> containerReport`




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



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


[GitHub] [ozone] avijayanhwx commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r527076699



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -190,16 +217,32 @@ void setShutdownGracefully() {
   public boolean getShutdownOnError() {
     return shutdownOnError;
   }
+
   /**
    * Adds the report to report queue.
    *
    * @param report report to be added
    */
   public void addReport(GeneratedMessage report) {
     if (report != null) {
-      synchronized (reports) {
-        for (InetSocketAddress endpoint : endpoints) {
-          reports.get(endpoint).add(report);
+      final String reportType = report.getDescriptorForType().getFullName();
+      for (InetSocketAddress endpoint : endpoints) {
+        // Check report type
+        if (reportType.equals(containerReportsProtoName)) {
+          containerReport = report;
+        } else if (reportType.equals(nodeReportProtoName)) {
+          nodeReport = report;
+        } else if (reportType.equals(pipelineReportsProtoName)) {
+          pipelineReport = report;
+        } else if (reportType.equals(commandStatusReportsProtoName) ||
+            reportType.equals(incrementalContainerReportProtoName)) {
+          // report type is CommandStatusReports or IncrementalContainerReport
+          synchronized (reports) {
+            reports.get(endpoint).add(report);

Review comment:
       I see that this is a new logical condition here. When a new report publisher is added, explicit changes are needed here. If not, we can keep Line 237 as a simple else.

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
##########
@@ -184,15 +181,30 @@ private void addReports(SCMHeartbeatRequestProto.Builder requestBuilder) {
     for (GeneratedMessage report :
         context.getAllAvailableReports(rpcEndpoint.getAddress())) {
       String reportName = report.getDescriptorForType().getFullName();
+      // Example reportName = hadoop.hdds.NodeReportProto
+
       for (Descriptors.FieldDescriptor descriptor :
           SCMHeartbeatRequestProto.getDescriptor().getFields()) {
+
         String heartbeatFieldName = descriptor.getMessageType().getFullName();
+        // Possible heartbeatFieldName =
+        //  hadoop.hdds.DatanodeDetailsProto
+        //  hadoop.hdds.NodeReportProto
+        //  hadoop.hdds.ContainerReportsProto
+        //  hadoop.hdds.IncrementalContainerReportProto
+        //  hadoop.hdds.CommandStatusReportsProto
+        //  hadoop.hdds.ContainerActionsProto
+        //  hadoop.hdds.PipelineActionsProto
+        //  hadoop.hdds.PipelineReportsProto
         if (heartbeatFieldName.equals(reportName)) {
           if (descriptor.isRepeated()) {
             requestBuilder.addRepeatedField(descriptor, report);
           } else {
             requestBuilder.setField(descriptor, report);
           }
+          // TODO: We can exit loop early here since we have a match already,
+          //  right? Double check.
+          break;

Review comment:
       Yes, it looks like we can break out 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



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


[GitHub] [ozone] runzhiwang edited a comment on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang edited a comment on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-731956707


   @smengcl Thanks the patch. It looks good to me overall. But I think we need more strong integration test to cover the case: StateContext#reports will not contain any ContainerReport, NodeReport, and DatanodeReport.


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



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


[GitHub] [ozone] smengcl commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r530596690



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -72,7 +101,13 @@
   private final AtomicLong stateExecutionCount;
   private final ConfigurationSource conf;
   private final Set<InetSocketAddress> endpoints;
-  private final Map<InetSocketAddress, List<GeneratedMessage>> reports;
+  // Only the latest full report of each type is kept
+  private final AtomicReference<GeneratedMessage> containerReports;

Review comment:
       Thanks for the tip. I will keep atomic for now as the report is not updated so frequently on DNs (low performance impact).




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



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


[GitHub] [ozone] runzhiwang commented on a change in pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#discussion_r528494451



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
##########
@@ -72,6 +76,11 @@
   private final AtomicLong stateExecutionCount;
   private final ConfigurationSource conf;
   private final Set<InetSocketAddress> endpoints;
+  // Only keeps the latest Container, Node and Pipeline report
+  private GeneratedMessage containerReports;

Review comment:
       containerReports -> containerReport




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



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


[GitHub] [ozone] runzhiwang commented on pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #1601:
URL: https://github.com/apache/ozone/pull/1601#issuecomment-744378432


   @smengcl Thanks the patch. @adoroszlai @avijayanhwx @GlenGeng Thanks for review, I have merged the patch.


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



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


[GitHub] [ozone] runzhiwang merged pull request #1601: HDDS-4404. Datanode can go OOM when a Recon or SCM Server is very slow in processing reports

Posted by GitBox <gi...@apache.org>.
runzhiwang merged pull request #1601:
URL: https://github.com/apache/ozone/pull/1601


   


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



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