|
21 | 21 | import jakarta.annotation.Nonnull; |
22 | 22 | import org.apache.hadoop.hdds.conf.OzoneConfiguration; |
23 | 23 | import org.apache.hadoop.hdds.protocol.DatanodeDetails; |
| 24 | +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; |
| 25 | +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; |
| 26 | +import org.apache.hadoop.hdds.scm.ContainerPlacementStatus; |
24 | 27 | import org.apache.hadoop.hdds.scm.container.ContainerID; |
| 28 | +import org.apache.hadoop.hdds.scm.container.ContainerInfo; |
| 29 | +import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; |
| 30 | +import org.apache.hadoop.hdds.scm.container.ContainerReplica; |
| 31 | +import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException; |
25 | 32 | import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo; |
| 33 | +import org.apache.hadoop.hdds.scm.node.NodeStatus; |
26 | 34 | import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; |
27 | 35 | import org.apache.hadoop.ozone.OzoneConsts; |
28 | 36 | import org.apache.ozone.test.GenericTestUtils; |
|
36 | 44 | import java.time.Duration; |
37 | 45 | import java.util.ArrayList; |
38 | 46 | import java.util.Arrays; |
| 47 | +import java.util.HashSet; |
39 | 48 | import java.util.List; |
| 49 | +import java.util.Map; |
| 50 | +import java.util.Set; |
40 | 51 | import java.util.concurrent.CompletableFuture; |
41 | 52 | import java.util.concurrent.TimeoutException; |
| 53 | +import java.util.stream.Collectors; |
42 | 54 | import java.util.stream.Stream; |
43 | 55 |
|
44 | 56 | import static org.apache.hadoop.hdds.scm.container.balancer.TestableCluster.RANDOM; |
45 | 57 | import static org.assertj.core.api.Assertions.assertThat; |
46 | 58 | import static org.junit.jupiter.api.Assertions.assertEquals; |
47 | 59 | import static org.junit.jupiter.api.Assertions.assertFalse; |
48 | 60 | import static org.junit.jupiter.api.Assertions.assertNotEquals; |
| 61 | +import static org.junit.jupiter.api.Assertions.assertSame; |
49 | 62 | import static org.junit.jupiter.api.Assertions.assertTrue; |
50 | 63 | import static org.mockito.ArgumentMatchers.any; |
51 | 64 | import static org.mockito.Mockito.atLeastOnce; |
@@ -255,14 +268,11 @@ public void testBalancerWithMoveManager(@Nonnull MockedSCM mockedSCM) |
255 | 268 | mockedSCM.disableLegacyReplicationManager(); |
256 | 269 | mockedSCM.startBalancerTask(config); |
257 | 270 |
|
258 | | - verify(mockedSCM.getMoveManager(), atLeastOnce()) |
259 | | - .move(any(ContainerID.class), |
260 | | - any(DatanodeDetails.class), |
261 | | - any(DatanodeDetails.class)); |
| 271 | + verify(mockedSCM.getMoveManager(), atLeastOnce()). |
| 272 | + move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); |
262 | 273 |
|
263 | 274 | verify(mockedSCM.getReplicationManager(), times(0)) |
264 | | - .move(any(ContainerID.class), any( |
265 | | - DatanodeDetails.class), any(DatanodeDetails.class)); |
| 275 | + .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); |
266 | 276 | } |
267 | 277 |
|
268 | 278 | @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
@@ -323,6 +333,261 @@ public void testMetrics(@Nonnull MockedSCM mockedSCM) throws IOException, NodeNo |
323 | 333 | assertEquals(1, metrics.getNumContainerMovesFailed()); |
324 | 334 | } |
325 | 335 |
|
| 336 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 337 | + @MethodSource("createMockedSCMs") |
| 338 | + public void containerBalancerShouldSelectOnlyClosedContainers(@Nonnull MockedSCM mockedSCM) { |
| 339 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 340 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 341 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 342 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 343 | + } |
| 344 | + config.setIterations(1); |
| 345 | + config.setThreshold(10); |
| 346 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 347 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 348 | + |
| 349 | + Map<ContainerID, ContainerInfo> cidToInfoMap = mockedSCM.getCluster().getCidToInfoMap(); |
| 350 | + // Make all containers open, balancer should not select any of them |
| 351 | + for (ContainerInfo containerInfo : cidToInfoMap.values()) { |
| 352 | + containerInfo.setState(HddsProtos.LifeCycleState.OPEN); |
| 353 | + } |
| 354 | + |
| 355 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 356 | + |
| 357 | + // Balancer should have identified unbalanced nodes |
| 358 | + assertFalse(TestContainerBalancerDatanodeNodeLimit.getUnBalancedNodes(task).isEmpty()); |
| 359 | + // No container should have been selected |
| 360 | + assertTrue(task.getContainerToSourceMap().isEmpty()); |
| 361 | + |
| 362 | + // Iteration result should be CAN_NOT_BALANCE_ANY_MORE because no container move is generated |
| 363 | + assertEquals(ContainerBalancerTask.IterationResult.CAN_NOT_BALANCE_ANY_MORE, task.getIterationResult()); |
| 364 | + |
| 365 | + // Now, close all containers |
| 366 | + for (ContainerInfo containerInfo : cidToInfoMap.values()) { |
| 367 | + containerInfo.setState(HddsProtos.LifeCycleState.CLOSED); |
| 368 | + } |
| 369 | + ContainerBalancerTask nextTask = mockedSCM.startBalancerTask(config); |
| 370 | + |
| 371 | + // Check whether all selected containers are closed |
| 372 | + for (ContainerID cid: nextTask.getContainerToSourceMap().keySet()) { |
| 373 | + assertSame(cidToInfoMap.get(cid).getState(), HddsProtos.LifeCycleState.CLOSED); |
| 374 | + } |
| 375 | + } |
| 376 | + |
| 377 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 378 | + @MethodSource("createMockedSCMs") |
| 379 | + public void balancerShouldNotSelectNonClosedContainerReplicas(@Nonnull MockedSCM mockedSCM) |
| 380 | + throws ContainerNotFoundException |
| 381 | + { |
| 382 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 383 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 384 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 385 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 386 | + } |
| 387 | + config.setIterations(1); |
| 388 | + config.setThreshold(10); |
| 389 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 390 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 391 | + |
| 392 | + // Let's mock such that all replicas have CLOSING state |
| 393 | + Map<ContainerID, Set<ContainerReplica>> cidToReplicasMap = mockedSCM.getCluster().getCidToReplicasMap(); |
| 394 | + when(mockedSCM.getContainerManager().getContainerReplicas(any(ContainerID.class))) |
| 395 | + .thenAnswer(invocationOnMock -> { |
| 396 | + ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0]; |
| 397 | + Set<ContainerReplica> replicas = cidToReplicasMap.get(cid); |
| 398 | + Set<ContainerReplica> replicasToReturn = new HashSet<>(replicas.size()); |
| 399 | + for (ContainerReplica replica : replicas) { |
| 400 | + ContainerReplica newReplica = replica.toBuilder() |
| 401 | + .setContainerState(StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSING) |
| 402 | + .build(); |
| 403 | + replicasToReturn.add(newReplica); |
| 404 | + } |
| 405 | + |
| 406 | + return replicasToReturn; |
| 407 | + }); |
| 408 | + |
| 409 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 410 | + |
| 411 | + // Balancer should have identified unbalanced nodes |
| 412 | + assertFalse(TestContainerBalancerDatanodeNodeLimit.getUnBalancedNodes(task).isEmpty()); |
| 413 | + // No container should have moved because all replicas are CLOSING |
| 414 | + assertTrue(task.getContainerToSourceMap().isEmpty()); |
| 415 | + } |
| 416 | + |
| 417 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 418 | + @MethodSource("createMockedSCMs") |
| 419 | + public void containerBalancerShouldObeyMaxSizeToMoveLimit(@Nonnull MockedSCM mockedSCM) { |
| 420 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 421 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 422 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 423 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 424 | + } |
| 425 | + config.setIterations(1); |
| 426 | + config.setThreshold(1); |
| 427 | + config.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT); |
| 428 | + config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); |
| 429 | + |
| 430 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 431 | + |
| 432 | + // Balancer should not have moved more size than the limit |
| 433 | + assertThat(task.getSizeScheduledForMoveInLatestIteration()).isLessThanOrEqualTo(10 * STORAGE_UNIT); |
| 434 | + |
| 435 | + long size = task.getMetrics().getDataSizeMovedGBInLatestIteration(); |
| 436 | + assertThat(size).isGreaterThan(0); |
| 437 | + assertThat(size).isLessThanOrEqualTo(10); |
| 438 | + } |
| 439 | + |
| 440 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 441 | + @MethodSource("createMockedSCMs") |
| 442 | + public void targetDatanodeShouldNotAlreadyContainSelectedContainer(@Nonnull MockedSCM mockedSCM) { |
| 443 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 444 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 445 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 446 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 447 | + } |
| 448 | + config.setIterations(1); |
| 449 | + config.setThreshold(10); |
| 450 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 451 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 452 | + |
| 453 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 454 | + |
| 455 | + Map<ContainerID, DatanodeDetails> map = task.getContainerToTargetMap(); |
| 456 | + Map<ContainerID, Set<ContainerReplica>> cidToReplicasMap = mockedSCM.getCluster().getCidToReplicasMap(); |
| 457 | + for (Map.Entry<ContainerID, DatanodeDetails> entry : map.entrySet()) { |
| 458 | + ContainerID container = entry.getKey(); |
| 459 | + DatanodeDetails target = entry.getValue(); |
| 460 | + assertTrue(cidToReplicasMap.get(container) |
| 461 | + .stream() |
| 462 | + .map(ContainerReplica::getDatanodeDetails) |
| 463 | + .noneMatch(target::equals)); |
| 464 | + } |
| 465 | + } |
| 466 | + |
| 467 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 468 | + @MethodSource("createMockedSCMs") |
| 469 | + public void containerMoveSelectionShouldFollowPlacementPolicy(@Nonnull MockedSCM mockedSCM) { |
| 470 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 471 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 472 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 473 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 474 | + } |
| 475 | + config.setIterations(1); |
| 476 | + config.setThreshold(10); |
| 477 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 478 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 479 | + |
| 480 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 481 | + |
| 482 | + Map<ContainerID, DatanodeDetails> containerFromSourceMap = task.getContainerToSourceMap(); |
| 483 | + Map<ContainerID, DatanodeDetails> containerToTargetMap = task.getContainerToTargetMap(); |
| 484 | + |
| 485 | + // For each move selection, check if {replicas - source + target} satisfies placement policy |
| 486 | + for (Map.Entry<ContainerID, DatanodeDetails> entry : containerFromSourceMap.entrySet()) { |
| 487 | + ContainerID container = entry.getKey(); |
| 488 | + DatanodeDetails source = entry.getValue(); |
| 489 | + |
| 490 | + List<DatanodeDetails> replicas = mockedSCM.getCluster().getCidToReplicasMap().get(container) |
| 491 | + .stream() |
| 492 | + .map(ContainerReplica::getDatanodeDetails) |
| 493 | + .collect(Collectors.toList()); |
| 494 | + // Remove source and add target |
| 495 | + replicas.remove(source); |
| 496 | + replicas.add(containerToTargetMap.get(container)); |
| 497 | + |
| 498 | + ContainerInfo containerInfo = mockedSCM.getCluster().getCidToInfoMap().get(container); |
| 499 | + ContainerPlacementStatus placementStatus; |
| 500 | + int requiredNodes = containerInfo.getReplicationConfig().getRequiredNodes(); |
| 501 | + if (containerInfo.getReplicationType() == HddsProtos.ReplicationType.RATIS) { |
| 502 | + placementStatus = mockedSCM.getPlacementPolicy().validateContainerPlacement(replicas, requiredNodes); |
| 503 | + } else { |
| 504 | + placementStatus = mockedSCM.getEcPlacementPolicy().validateContainerPlacement(replicas, requiredNodes); |
| 505 | + } |
| 506 | + assertTrue(placementStatus.isPolicySatisfied()); |
| 507 | + } |
| 508 | + } |
| 509 | + |
| 510 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 511 | + @MethodSource("createMockedSCMs") |
| 512 | + public void targetDatanodeShouldBeInServiceHealthy(@Nonnull MockedSCM mockedSCM) throws NodeNotFoundException { |
| 513 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 514 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 515 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 516 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 517 | + } |
| 518 | + config.setIterations(1); |
| 519 | + config.setThreshold(10); |
| 520 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 521 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 522 | + |
| 523 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 524 | + |
| 525 | + for (DatanodeDetails target : task.getSelectedTargets()) { |
| 526 | + NodeStatus status = mockedSCM.getNodeManager().getNodeStatus(target); |
| 527 | + assertSame(HddsProtos.NodeOperationalState.IN_SERVICE, status.getOperationalState()); |
| 528 | + assertTrue(status.isHealthy()); |
| 529 | + } |
| 530 | + } |
| 531 | + |
| 532 | + |
| 533 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 534 | + @MethodSource("createMockedSCMs") |
| 535 | + public void selectedContainerShouldNotAlreadyHaveBeenSelected(@Nonnull MockedSCM mockedSCM) |
| 536 | + throws NodeNotFoundException, ContainerNotFoundException, TimeoutException, ContainerReplicaNotFoundException |
| 537 | + { |
| 538 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 539 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 540 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 541 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 542 | + } |
| 543 | + config.setIterations(1); |
| 544 | + config.setThreshold(10); |
| 545 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 546 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 547 | + |
| 548 | + mockedSCM.enableLegacyReplicationManager(); |
| 549 | + |
| 550 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 551 | + int numContainers = task.getContainerToTargetMap().size(); |
| 552 | + |
| 553 | + /* Assuming move is called exactly once for each unique container, number of calls to move should equal number of |
| 554 | + unique containers. If number of calls to move is more than number of unique containers, at least one container |
| 555 | + has been re-selected. It's expected that number of calls to move should equal number of unique, selected containers |
| 556 | + (from containerToTargetMap). |
| 557 | + */ |
| 558 | + verify(mockedSCM.getReplicationManager(), times(numContainers)) |
| 559 | + .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); |
| 560 | + |
| 561 | + // Try the same test by disabling LegacyReplicationManager so that MoveManager is used. |
| 562 | + mockedSCM.disableLegacyReplicationManager(); |
| 563 | + ContainerBalancerTask nextTask = mockedSCM.startBalancerTask(config); |
| 564 | + |
| 565 | + numContainers = nextTask.getContainerToTargetMap().size(); |
| 566 | + verify(mockedSCM.getMoveManager(), times(numContainers)) |
| 567 | + .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); |
| 568 | + } |
| 569 | + |
| 570 | + @ParameterizedTest(name = "MockedSCM #{index}: {0}") |
| 571 | + @MethodSource("createMockedSCMs") |
| 572 | + public void balancerShouldNotSelectConfiguredExcludeContainers(@Nonnull MockedSCM mockedSCM) { |
| 573 | + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new OzoneConfiguration()); |
| 574 | + int nodeCount = mockedSCM.getCluster().getNodeCount(); |
| 575 | + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { |
| 576 | + config.setMaxDatanodesPercentageToInvolvePerIteration(100); |
| 577 | + } |
| 578 | + config.setIterations(1); |
| 579 | + config.setThreshold(10); |
| 580 | + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); |
| 581 | + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); |
| 582 | + config.setExcludeContainers("1, 4, 5"); |
| 583 | + |
| 584 | + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); |
| 585 | + |
| 586 | + Set<ContainerID> excludeContainers = config.getExcludeContainers(); |
| 587 | + for (ContainerID container : task.getContainerToSourceMap().keySet()) { |
| 588 | + assertThat(excludeContainers).doesNotContain(container); |
| 589 | + } |
| 590 | + } |
326 | 591 |
|
327 | 592 | public static List<DatanodeUsageInfo> getUnBalancedNodes(@Nonnull ContainerBalancerTask task) { |
328 | 593 | ArrayList<DatanodeUsageInfo> result = new ArrayList<>(); |
|
0 commit comments