浏览代码

HDDS-2081. Fix TestRatisPipelineProvider#testCreatePipelinesDnExclude. Contributed by Aravindan Vijayan. (#1506)

avijayanhwx 5 年之前
父节点
当前提交
8f1a135118

+ 16 - 10
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java

@@ -139,9 +139,15 @@ public class TestRatisPipelineProvider {
   @Test
   public void testCreatePipelinesDnExclude() throws IOException {
 
-    // We have 10 DNs in MockNodeManager.
+    // We need 9 Healthy DNs in MockNodeManager.
+    NodeManager mockNodeManager = new MockNodeManager(true, 12);
+    PipelineStateManager stateManagerMock =
+        new PipelineStateManager(new OzoneConfiguration());
+    PipelineProvider providerMock = new MockRatisPipelineProvider(
+        mockNodeManager, stateManagerMock, new OzoneConfiguration());
+
     // Use up first 3 DNs for an open pipeline.
-    List<DatanodeDetails> openPiplineDns = nodeManager.getAllNodes()
+    List<DatanodeDetails> openPiplineDns = mockNodeManager.getAllNodes()
         .subList(0, 3);
     HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE;
 
@@ -153,10 +159,10 @@ public class TestRatisPipelineProvider {
         .setId(PipelineID.randomId())
         .build();
 
-    stateManager.addPipeline(openPipeline);
+    stateManagerMock.addPipeline(openPipeline);
 
     // Use up next 3 DNs also for an open pipeline.
-    List<DatanodeDetails> moreOpenPiplineDns = nodeManager.getAllNodes()
+    List<DatanodeDetails> moreOpenPiplineDns = mockNodeManager.getAllNodes()
         .subList(3, 6);
     Pipeline anotherOpenPipeline = Pipeline.newBuilder()
         .setType(HddsProtos.ReplicationType.RATIS)
@@ -165,10 +171,10 @@ public class TestRatisPipelineProvider {
         .setState(Pipeline.PipelineState.OPEN)
         .setId(PipelineID.randomId())
         .build();
-    stateManager.addPipeline(anotherOpenPipeline);
+    stateManagerMock.addPipeline(anotherOpenPipeline);
 
     // Use up next 3 DNs also for a closed pipeline.
-    List<DatanodeDetails> closedPiplineDns = nodeManager.getAllNodes()
+    List<DatanodeDetails> closedPiplineDns = mockNodeManager.getAllNodes()
         .subList(6, 9);
     Pipeline anotherClosedPipeline = Pipeline.newBuilder()
         .setType(HddsProtos.ReplicationType.RATIS)
@@ -177,9 +183,9 @@ public class TestRatisPipelineProvider {
         .setState(Pipeline.PipelineState.CLOSED)
         .setId(PipelineID.randomId())
         .build();
-    stateManager.addPipeline(anotherClosedPipeline);
+    stateManagerMock.addPipeline(anotherClosedPipeline);
 
-    Pipeline pipeline = provider.create(factor);
+    Pipeline pipeline = providerMock.create(factor);
     Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS);
     Assert.assertEquals(pipeline.getFactor(), factor);
     Assert.assertEquals(pipeline.getPipelineState(),
@@ -193,8 +199,8 @@ public class TestRatisPipelineProvider {
         (openPiplineDns.contains(dn) || moreOpenPiplineDns.contains(dn)))
         .count() == 0);
 
-    // Since we have only 10 DNs, at least 1 pipeline node should have been
-    // from the closed pipeline DN list.
+    // Since we have only 9 Healthy DNs, at least 1 pipeline node should have
+    // been from the closed pipeline DN list.
     Assert.assertTrue(pipelineNodes.parallelStream().filter(
         closedPiplineDns::contains).count() > 0);
   }