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 2022/06/26 06:15:07 UTC

[GitHub] [ozone] umamaheswararao opened a new pull request, #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

umamaheswararao opened a new pull request, #3550:
URL: https://github.com/apache/ozone/pull/3550

   ## What changes were proposed in this pull request?
   
   This JIRA addresses the issue regarding DN reconstruction command queues reporting. Now with this changes, when reconstruction command received by DN, while executing the task, counter will be incremented and after execution of the coordinator task counter will be reduced. 
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6945
   
   ## How was this patch tested?
   
   added test.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] sodonnel commented on a diff in pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3550:
URL: https://github.com/apache/ozone/pull/3550#discussion_r907263321


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionSupervisor.java:
##########
@@ -38,6 +39,11 @@ public class ECReconstructionSupervisor implements Closeable {
   private final StateContext context;
   private final ExecutorService executor;
   private final ECReconstructionCoordinator reconstructionCoordinator;
+  /**
+   * how many coordinator tasks currently being running.
+   */
+  private final ConcurrentHashMap.KeySetView<Object, Boolean>

Review Comment:
   These seems to be copied from the ReplicationSupervisor - do we have a plan to use this for more than just counting tasks? Could we use an atomicLong as a simple counter instead?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionSupervisor.java:
##########
@@ -38,6 +39,11 @@ public class ECReconstructionSupervisor implements Closeable {
   private final StateContext context;
   private final ExecutorService executor;
   private final ECReconstructionCoordinator reconstructionCoordinator;
+  /**
+   * how many coordinator tasks currently being running.
+   */
+  private final ConcurrentHashMap.KeySetView<Object, Boolean>

Review Comment:
   This seems to be copied from the ReplicationSupervisor - do we have a plan to use this for more than just counting tasks? Could we use an atomicLong as a simple counter instead?



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] umamaheswararao commented on pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on PR #3550:
URL: https://github.com/apache/ozone/pull/3550#issuecomment-1168530263

   Thanks @sodonnel for the reviews!


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] umamaheswararao merged pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
umamaheswararao merged PR #3550:
URL: https://github.com/apache/ozone/pull/3550


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] umamaheswararao commented on a diff in pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on code in PR #3550:
URL: https://github.com/apache/ozone/pull/3550#discussion_r907473592


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionSupervisor.java:
##########
@@ -38,6 +39,11 @@ public class ECReconstructionSupervisor implements Closeable {
   private final StateContext context;
   private final ExecutorService executor;
   private final ECReconstructionCoordinator reconstructionCoordinator;
+  /**
+   * how many coordinator tasks currently being running.
+   */
+  private final ConcurrentHashMap.KeySetView<Object, Boolean>

Review Comment:
   Yes, Just used this from ReplicationSupervisor. I thought is atomicLong, but looks like we are trying to avoid duplicate task for same command by any chance.
   
   While adding task:
   if (inProgressReconstrucionCoordinatorCounter
           .add(taskInfo.getContainerID())) {
           
    I thought this may be useful check for safety and may avoid any accidental corruption due to fight two containers. Ideally that should be protected as we would create containers, the other one should fail.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] umamaheswararao commented on pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on PR #3550:
URL: https://github.com/apache/ozone/pull/3550#issuecomment-1167452020

   I attempted to remove that. I figured out more classes complained to implement that.
   ![image](https://user-images.githubusercontent.com/1719507/175969392-0e06b0f5-e37c-4cd1-811d-03e96ed467d2.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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] sodonnel commented on a diff in pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3550:
URL: https://github.com/apache/ozone/pull/3550#discussion_r907261873


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ec/reconstruction/TestECReconstructionSupervisor.java:
##########
@@ -17,37 +17,57 @@
  */
 package org.apache.hadoop.ozone.container.ec.reconstruction;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.SortedMap;
 import java.util.concurrent.TimeoutException;
 
 /**
  * Tests the ECReconstructionSupervisor.
  */
 public class TestECReconstructionSupervisor {
 
-  private final ECReconstructionSupervisor supervisor =
-      new ECReconstructionSupervisor(null, null, 5, null);
-
   @Test
   public void testAddTaskShouldExecuteTheGivenTask()
-      throws InterruptedException, TimeoutException {
-    FakeTask task = new FakeTask(null);
-    supervisor.addTask(task);
-    GenericTestUtils.waitFor(() -> task.isExecuted, 100, 15000);
-  }
-
-  static class FakeTask extends ECReconstructionCoordinatorTask {
-    private boolean isExecuted = false;
-
-    FakeTask(ECReconstructionCommandInfo reconstructionCommandInfo) {
-      super(null, reconstructionCommandInfo);
-    }
-
-    @Override
-    public void run() {
-      isExecuted = true;
-    }
+      throws InterruptedException, TimeoutException, IOException {
+    final boolean[] reconstructInvoked = {false};

Review Comment:
   Why are these arrays? I guess its because a Boolean is immutable and we cannot change it if its a final, and the variable has to be final to be use in the run block.
   
   I think it would be more clear to use a pair of CountDownLatch objects here. Then we can get rid of most of the waitFor() calls, eg
   
   ```
   CountDownLatch runnableInvoked = new CountDownLatch(1);
   CountDownLatch holdProcessing = new CountDownLatch(1);
   
   // Inside the reconstructECContainerGroup
   ...
   runnableInvoked.countDown();
   holdProcessing.await();
   ...
   
   // In the main test method:
       supervisor.addTask(
           new ECReconstructionCommandInfo(1, new ECReplicationConfig(3, 2),
               new byte[0], ImmutableList.of(), ImmutableList.of()));
       runnableInvoked.await()
       Assert.assertEquals(1, supervisor.getInFlightReplications());
       holdProcessing.countDown();
       GenericTestUtils
           .waitFor(() -> supervisor.getInFlightReplications() == 0, 100, 15000);
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] umamaheswararao commented on a diff in pull request #3550: HDDS-6945. EC: EC Reconstruction Command count queues should be included in DN heartbeat

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on code in PR #3550:
URL: https://github.com/apache/ozone/pull/3550#discussion_r907467318


##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ec/reconstruction/TestECReconstructionSupervisor.java:
##########
@@ -17,37 +17,57 @@
  */
 package org.apache.hadoop.ozone.container.ec.reconstruction;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.SortedMap;
 import java.util.concurrent.TimeoutException;
 
 /**
  * Tests the ECReconstructionSupervisor.
  */
 public class TestECReconstructionSupervisor {
 
-  private final ECReconstructionSupervisor supervisor =
-      new ECReconstructionSupervisor(null, null, 5, null);
-
   @Test
   public void testAddTaskShouldExecuteTheGivenTask()
-      throws InterruptedException, TimeoutException {
-    FakeTask task = new FakeTask(null);
-    supervisor.addTask(task);
-    GenericTestUtils.waitFor(() -> task.isExecuted, 100, 15000);
-  }
-
-  static class FakeTask extends ECReconstructionCoordinatorTask {
-    private boolean isExecuted = false;
-
-    FakeTask(ECReconstructionCommandInfo reconstructionCommandInfo) {
-      super(null, reconstructionCommandInfo);
-    }
-
-    @Override
-    public void run() {
-      isExecuted = true;
-    }
+      throws InterruptedException, TimeoutException, IOException {
+    final boolean[] reconstructInvoked = {false};

Review Comment:
   Ah it was just intelliJ sggestion :-)
   Thanks for suggesting CountDownLatch, it's a good fit here.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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