You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "adoroszlai (via GitHub)" <gi...@apache.org> on 2024/01/17 17:25:25 UTC

Re: [PR] HDDS-9345. Add CapacityPipelineChoosePolicy considering datanode storage space [ozone]

adoroszlai commented on code in PR #5354:
URL: https://github.com/apache/ozone/pull/5354#discussion_r1455996846


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestCapacityPipelineChoosePolicy.java:
##########
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.pipeline.choose.algorithms;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
+import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for the capacity pipeline choose policy.
+ */
+public class TestCapacityPipelineChoosePolicy {
+  @Test
+  public void choosePipeline() throws Exception {
+
+    // given 4 datanode
+    List<DatanodeDetails> datanodes = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
+    }
+    //          dn0   dn1   dn2   dn3
+    // used       0   10    20    30
+    NodeManager mockNodeManager = Mockito.mock(NodeManager.class);
+    when(mockNodeManager.getNodeStat(datanodes.get(0)))
+        .thenReturn(new SCMNodeMetric(100L, 0L, 100L));
+    when(mockNodeManager.getNodeStat(datanodes.get(1)))
+        .thenReturn(new SCMNodeMetric(100L, 10L, 90L));
+    when(mockNodeManager.getNodeStat(datanodes.get(2)))
+        .thenReturn(new SCMNodeMetric(100L, 20L, 80L));
+    when(mockNodeManager.getNodeStat(datanodes.get(3)))
+        .thenReturn(new SCMNodeMetric(100L, 30L, 70L));
+
+    CapacityPipelineChoosePolicy policy =
+        new CapacityPipelineChoosePolicy(mockNodeManager);
+
+    // generate 4 pipelines, and every pipeline has 3 datanodes
+    //
+    //  pipeline0    dn1   dn2   dn3
+    //  pipeline1    dn0   dn2   dn3
+    //  pipeline2    dn0   dn1   dn3
+    //  pipeline3    dn0   dn1   dn2
+    //
+    // In the above scenario, pipeline0 vs pipeline1 runs through three rounds
+    // of comparisons, (dn3 <-> dn3) -> (dn2 <-> dn2 ) -> (dn1 <-> dn0),
+    // finally comparing dn0 and dn1, and dn0 wins, so pipeline1 is selected.
+    //
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      List<DatanodeDetails> dns = new ArrayList<>();
+      for (int j = 0; j < datanodes.size(); j++) {
+        if (i != j) {
+          dns.add(datanodes.get(j));
+        }
+      }
+      Pipeline pipeline = MockPipeline.createPipeline(dns);
+      MockRatisPipelineProvider.markPipelineHealthy(pipeline);
+      pipelines.add(pipeline);
+    }
+
+    Map<Pipeline, Integer> selectedCount = new HashMap<>();
+    for (Pipeline pipeline : pipelines) {
+      selectedCount.put(pipeline, 0);
+    }
+    for (int i = 0; i < 1000; i++) {
+      // choosePipeline
+      Pipeline pipeline = policy.choosePipeline(pipelines, null);
+      Assertions.assertNotNull(pipeline);

Review Comment:
   nit: please add static imports
   
   ```suggestion
         assertNotNull(pipeline);
   ```



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestCapacityPipelineChoosePolicy.java:
##########
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.pipeline.choose.algorithms;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
+import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for the capacity pipeline choose policy.
+ */
+public class TestCapacityPipelineChoosePolicy {
+  @Test
+  public void choosePipeline() throws Exception {
+
+    // given 4 datanode
+    List<DatanodeDetails> datanodes = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
+    }
+    //          dn0   dn1   dn2   dn3
+    // used       0   10    20    30
+    NodeManager mockNodeManager = Mockito.mock(NodeManager.class);
+    when(mockNodeManager.getNodeStat(datanodes.get(0)))
+        .thenReturn(new SCMNodeMetric(100L, 0L, 100L));
+    when(mockNodeManager.getNodeStat(datanodes.get(1)))
+        .thenReturn(new SCMNodeMetric(100L, 10L, 90L));
+    when(mockNodeManager.getNodeStat(datanodes.get(2)))
+        .thenReturn(new SCMNodeMetric(100L, 20L, 80L));
+    when(mockNodeManager.getNodeStat(datanodes.get(3)))
+        .thenReturn(new SCMNodeMetric(100L, 30L, 70L));
+
+    CapacityPipelineChoosePolicy policy =
+        new CapacityPipelineChoosePolicy(mockNodeManager);
+
+    // generate 4 pipelines, and every pipeline has 3 datanodes
+    //
+    //  pipeline0    dn1   dn2   dn3
+    //  pipeline1    dn0   dn2   dn3
+    //  pipeline2    dn0   dn1   dn3
+    //  pipeline3    dn0   dn1   dn2
+    //
+    // In the above scenario, pipeline0 vs pipeline1 runs through three rounds
+    // of comparisons, (dn3 <-> dn3) -> (dn2 <-> dn2 ) -> (dn1 <-> dn0),
+    // finally comparing dn0 and dn1, and dn0 wins, so pipeline1 is selected.
+    //
+    List<Pipeline> pipelines = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      List<DatanodeDetails> dns = new ArrayList<>();
+      for (int j = 0; j < datanodes.size(); j++) {
+        if (i != j) {
+          dns.add(datanodes.get(j));
+        }
+      }
+      Pipeline pipeline = MockPipeline.createPipeline(dns);
+      MockRatisPipelineProvider.markPipelineHealthy(pipeline);
+      pipelines.add(pipeline);
+    }
+
+    Map<Pipeline, Integer> selectedCount = new HashMap<>();
+    for (Pipeline pipeline : pipelines) {
+      selectedCount.put(pipeline, 0);
+    }
+    for (int i = 0; i < 1000; i++) {
+      // choosePipeline
+      Pipeline pipeline = policy.choosePipeline(pipelines, null);
+      Assertions.assertNotNull(pipeline);
+      selectedCount.put(pipeline, selectedCount.get(pipeline) + 1);
+    }
+
+    // The selected count from most to least should be :
+    // pipeline3 > pipeline2 > pipeline1 > pipeline0
+    for (int i = 0; i < pipelines.size(); i++) {
+      System.out.println("pipeline" + i + " selected count: "
+          + selectedCount.get(pipelines.get(i)));
+    }
+    Assertions.assertTrue(selectedCount.get(pipelines.get(3))
+        > selectedCount.get(pipelines.get(2)));

Review Comment:
   nit: please use AssertJ for better failure message instead of using std. output.
   
   ```suggestion
       assertThat(selectedCount.get(pipelines.get(3)))
           .isGreaterThan(selectedCount.get(pipelines.get(2)));
   ```
   
   + `import static org.assertj.core.api.Assertions.assertThat`
   + same for the other two assertions below



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestCapacityPipelineChoosePolicy.java:
##########
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.pipeline.choose.algorithms;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
+import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for the capacity pipeline choose policy.
+ */
+public class TestCapacityPipelineChoosePolicy {
+  @Test
+  public void choosePipeline() throws Exception {
+
+    // given 4 datanode
+    List<DatanodeDetails> datanodes = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      datanodes.add(MockDatanodeDetails.randomDatanodeDetails());
+    }
+    //          dn0   dn1   dn2   dn3
+    // used       0   10    20    30
+    NodeManager mockNodeManager = Mockito.mock(NodeManager.class);

Review Comment:
   nit: please add static import
   
   ```suggestion
       NodeManager mockNodeManager = mock(NodeManager.class);
   ```



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/RandomPipelineChoosePolicy.java:
##########
@@ -30,6 +31,9 @@
  */
 public class RandomPipelineChoosePolicy implements PipelineChoosePolicy {
 
+  public RandomPipelineChoosePolicy(NodeManager nodeManager) {
+  }

Review Comment:
   In addition to compatibility, falling back to the no-arg constructor would also let us avoid adding this.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/PipelineChoosePolicyFactory.java:
##########
@@ -64,17 +65,18 @@ public static PipelineChoosePolicy getPolicy(
         LOG.error("Met an exception while create pipeline choose policy "
             + "for the given class {}. Fallback to the default pipeline "
             + " choose policy {}", policyName, defaultPolicy, e);
-        return createPipelineChoosePolicyFromClass(defaultPolicy);
+        return createPipelineChoosePolicyFromClass(nodeManager, defaultPolicy);
       }
       throw e;
     }
   }
 
   private static PipelineChoosePolicy createPipelineChoosePolicyFromClass(
+      final NodeManager nodeManager,
       Class<? extends PipelineChoosePolicy> policyClass) throws SCMException {
     Constructor<? extends PipelineChoosePolicy> constructor;
     try {
-      constructor = policyClass.getDeclaredConstructor();
+      constructor = policyClass.getDeclaredConstructor(NodeManager.class);

Review Comment:
   I think we should make the change backwards compatible, by using no-args constructor if this one fails.  Although it's not very likely, there may be custom implementations using the existing contract.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/HealthyPipelineChoosePolicy.java:
##########
@@ -30,7 +31,11 @@
  */
 public class HealthyPipelineChoosePolicy implements PipelineChoosePolicy {
 
-  private PipelineChoosePolicy randomPolicy = new RandomPipelineChoosePolicy();
+  private PipelineChoosePolicy randomPolicy;
+
+  public HealthyPipelineChoosePolicy(NodeManager nodeManager) {
+    randomPolicy = new RandomPipelineChoosePolicy(nodeManager);
+  }

Review Comment:
   In addition to compatibility, falling back to the no-arg constructor would also let us avoid adding 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.

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