-
Notifications
You must be signed in to change notification settings - Fork 593
HDDS-4929. Select target datanodes and containers to move for Container Balancer #2441
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
# Conflicts: # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java # hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
# Conflicts: # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java # hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
|
@lokeshj1703 @JacksonYao287 @ChenSammi please take a look! |
|
thanks @siddhantsangwan for this work, can you please fix the CI first? |
|
@JacksonYao287 it seems |
|
please merge the latest master branch and try again, #2446 seems may the test case not flaky |
|
@JacksonYao287 it can be reproduced on |
@adoroszlai thank for pointing out this , i will fix this flaky case in that jira |
JacksonYao287
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for the work, i have some comments inline initially, please take a look. because code is too much , i will take a deeper review later in next week.
| this.idleIteration = config.getIdleIteration(); | ||
| this.threshold = config.getThreshold(); | ||
| this.maxDatanodesToBalance = config.getMaxDatanodesToBalance(); | ||
| this.maxSizeToMove = config.getMaxSizeToMove(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
move these code out of the loop, and for is preferred for loop
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
idleIteration should be moved out, but the rest should stay inside the loop, right? threshold and other parameters can be changed while balancer is running. So their values should be checked before every iteration.
| ContainerBalancerConfiguration balancerConfiguration) { | ||
| lock.lock(); | ||
| try { | ||
| if (!balancerRunning.compareAndSet(false, true)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
before starting balancer thread, we need to check whether the current scm is out of safe mode and is ratis leader?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This check was moved to the initializeIteration method as per this comment in #2230
| */ | ||
| public boolean start( | ||
| ContainerBalancerConfiguration balancerConfiguration) { | ||
| lock.lock(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
seems we do not need a lock here. balancerRunning.compareAndSet can take the same effect for avoid current balancer thread
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense, changed this.
| this.overUtilizedNodes = new ArrayList<>(); | ||
| this.underUtilizedNodes = new ArrayList<>(); | ||
| this.withinThresholdUtilizedNodes = new ArrayList<>(); | ||
| this.unBalancedNodes = new ArrayList<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove this
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where do you suggest these lists should be initialized then?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
clusterCapacity, clusterUsed and clusterRemaining will be renewed in calculateAvgUtilization at every iteration, seems no need to set the to 0 here specially.
and other variables can be renewed using clear() , no need to create a new object every time
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since one purpose of intitalizeIteration is to initialize the fields necessary for balancing, it seems clearer to initialize these values here instead of calculateAvgUtilization. Also since those lists need to be initialized at least once, we need to create them here. Instead, if this initialization is moved to the constructor, we will need to construct every time we want to restart balancer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also since those lists need to be initialized at least once, we need to create them here. Instead, if this initialization is moved to the constructor, we will need to construct every time we want to restart balancer.
maybe we can construct these list at the constructor function of balancer, and then reuse them with clear()
| ///////////////////////////// | ||
| } | ||
|
|
||
| LOG.info("Container Balancer has identified Datanodes that need to be" + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
better to log how many overUtilizedNodes and underUtilizedNodes also
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done!
| * Checks if limits maxDatanodesToBalance and maxSizeToMove have not been hit. | ||
| * @return true if conditions pass and balancing can continue, else false | ||
| */ | ||
| private boolean checkConditionsForBalancing() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe. we also. need to check is scm Leader now?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems the method name "checkConditionsForBalancing" is a bit confusing. Here, it's only checking the maxDatanodesToInvolvePerIteration and maxSizeToMovePerIteration limits. We can probably move this method to the Iteration class when it's introduced in HDDS-5518.
| return false; | ||
| } | ||
| return index >= 0 && listToSearch.get(index).equals(node); | ||
| if (totalSizeMoved + (long) ozoneConfiguration.getStorageSize( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
suppose totalSizeMoved = 97G ,maxSizeToMove = 100G , and we have a closed 2G ( < default 5G) container replica, can we move it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
According to this implementation, no. I've done this to avoid searching for a target datanode and container (and fail early) in case the maximum limit is about to be reached. Alternatively, we can perform this check after getting a ContainerMoveSelection.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can check container size directly?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, if we check after selecting a target datanode and container. In that case, some extra work would be performed if the check eventually fails.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Created a jira for this issue: https://issues.apache.org/jira/browse/HDDS-5526
|
|
||
| this.clusterCapacity = 0L; | ||
| this.clusterUsed = 0L; | ||
| this.clusterRemaining = 0L; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove clusterRemaining in current code , seems not used
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Okay!
| datanodeUsageInfo.getDatanodeDetails().getUuidString(), utilization); | ||
| if (utilization > upperLimit) { | ||
| overUtilizedNodes.add(datanodeUsageInfo); | ||
| countDatanodesToBalance += 1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove countDatanodesToBalance , overLoadedBytes, underLoadedBytes , no use here
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
canSizeLeaveSource, getClusterAvgUtilisation seem not used now, we can remove them
| moveSelection.getContainerID().toString(), source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString()); | ||
|
|
||
| // move container |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should call replicationManager#move here??
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I will add that in another commit
lokeshj1703
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@siddhantsangwan Thanks for working on the PR! The changes look really good. I have a few commnts inline.
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Show resolved
Hide resolved
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Outdated
Show resolved
Hide resolved
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Outdated
Show resolved
Hide resolved
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Outdated
Show resolved
Hide resolved
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
Outdated
Show resolved
Hide resolved
...n/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java
Show resolved
Hide resolved
...r-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
Outdated
Show resolved
Hide resolved
...r-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
Show resolved
Hide resolved
Refactor container.balancer package to change maxDatanodesToBalance to maxDatanodesToInvolvePerIteration.
# Conflicts: # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
|
@lokeshj1703 @JacksonYao287 Thanks for your reviews. I have integrated with |
…. Resolve comments. Refactor ContainerBalancerConfiguration#maxDatanodesToBalance and ContainerBalancerConfiguration#maxSizeToMove to maxDatanodesToInvolvePerIterationand maxSizeToMovePerIteration, respectively.
# Conflicts: # hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
|
@JacksonYao287 @lokeshj1703 I've addressed your reviews and created some new jiras wherever required. |
JacksonYao287
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks @siddhantsangwan for updating this PR. the changes looks good. i have some new comments inline, please take a look
| //we should start a new balancer thread async | ||
| //and response to cli as soon as possible | ||
| public boolean start(ContainerBalancerConfiguration balancerConfiguration) { | ||
| if (!balancerRunning.compareAndSet(false, true)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
after a deep thought , i think we may add the lock back. although compareAndSet can avoid concurrent start and concurrent stop , but it could not avoid concurrent start and stop. so i think the solution here is that, we use a lock to protect start and stop option , and thus balancerRunning is also protected by this lock ,so we can make balancerRunning boolen volitile , which is good too for isBalancerRunning
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense, changing this.
| private Map<DatanodeDetails, Long> sizeEnteringNode; | ||
| private Set<ContainerID> selectedContainers; | ||
| private FindTargetStrategy findTargetStrategy; | ||
| private PlacementPolicy placementPolicy; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
placementPolicy seems not used
| this.threshold = config.getThreshold(); | ||
| this.maxDatanodesToInvolvePerIteration = | ||
| config.getMaxDatanodesToInvolvePerIteration(); | ||
| this.maxSizeToMovePerIteration = config.getMaxSizeToMovePerIteration(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
as you mentioned, i am not sure why these parameters can be changed while balancer is running. i think they are set once in each balance thread. please correct me if i am wrong
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You're right. Balancer can be stopped and restarted if these parameters need to be edited. I'll change this.
| this.overUtilizedNodes = new ArrayList<>(); | ||
| this.underUtilizedNodes = new ArrayList<>(); | ||
| this.withinThresholdUtilizedNodes = new ArrayList<>(); | ||
| this.unBalancedNodes = new ArrayList<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also since those lists need to be initialized at least once, we need to create them here. Instead, if this initialization is moved to the constructor, we will need to construct every time we want to restart balancer.
maybe we can construct these list at the constructor function of balancer, and then reuse them with clear()
| // wait for configured time before starting next iteration, unless | ||
| // this was the final iteration | ||
| if (i != idleIteration - 1) { | ||
| synchronized (this) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why we need synchronized here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The wait method requires that the current thread owns this object's monitor
| return false; | ||
| private IterationResult checkConditionsForBalancing() { | ||
| if (countDatanodesInvolvedPerIteration + 2 > | ||
| maxDatanodesToInvolvePerIteration * totalNodesInCluster) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
better to rename maxDatanodesToInvolvePerIteration to maxDatanodesRatioToInvolvePerIteration, it is a little confused
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
| if (replicas.stream().noneMatch( | ||
| replica -> replica.getDatanodeDetails().equals(target)) && | ||
| containerMoveSatisfiesPlacementPolicy(container, replicas, source, | ||
| target) && | ||
| canSizeEnterTarget.apply(target, containerInfo.getUsedBytes())) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe later we can take networkTopology in to account, for all the candidate targets , we always choose the nearest one in networkTopology. this can be done in a new jira, maybe i can help doing this
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, we can do this in another jira.
lokeshj1703
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@siddhantsangwan Thanks for updating the PR! The changes look good to me. I have a few minor comments. Test failure seems related.
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Outdated
Show resolved
Hide resolved
...erver-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
Show resolved
Hide resolved
.../main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
Outdated
Show resolved
Hide resolved
| Thread.sleep(1000); | ||
| } catch (InterruptedException e) {} | ||
|
|
||
| Assert.assertFalse( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NIT: We could also add assertions for metrics here. Can be done in a separate PR.
|
It would also be a good idea to print the number of datanodes involved and total size moved after an iteration. |
|
It looks like |
| containerBalancerClient.startContainerBalancer(threshold, idleiterations, | ||
| maxDatanodesToBalance, maxSizeToMoveInGB); | ||
| maxDatanodesRatioToInvolvePerIteration, maxSizeToMovePerIterationInGB); | ||
| running = containerBalancerClient.getContainerBalancerStatus(); | ||
| assertTrue(running); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Balancing completes and exits before asserting that balancer is running because no unbalanced nodes are found.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes , that is a temporary implementation before balancer is fully completed and it is just for testing cli command, so it is may not Compatible with current balancer logic. could you also fix it in this PR?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if you could not fix it in this PR, please ignore or remove this test case for now , i will fix and add it back after this patch is merged
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I'm ignoring it for now
JacksonYao287
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks @siddhantsangwan for updating this PR. it looks good and close to merge. i have some minor comments inline,
| datanodeUsageInfo.getDatanodeDetails().getUuidString(), utilization); | ||
| if (utilization > upperLimit) { | ||
| overUtilizedNodes.add(datanodeUsageInfo); | ||
| countDatanodesToBalance += 1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
canSizeLeaveSource, getClusterAvgUtilisation seem not used now, we can remove them
| withinThresholdUtilizedNodes.size()); | ||
|
|
||
| // initialize maps to track how much size is leaving and entering datanodes | ||
| sizeLeavingNode = new HashMap<>(overUtilizedNodes.size() + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
these several maps can also be initialized at the constructor function and reused with clear
| containerBalancerClient.startContainerBalancer(threshold, idleiterations, | ||
| maxDatanodesToBalance, maxSizeToMoveInGB); | ||
| maxDatanodesRatioToInvolvePerIteration, maxSizeToMovePerIterationInGB); | ||
| running = containerBalancerClient.getContainerBalancerStatus(); | ||
| assertTrue(running); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes , that is a temporary implementation before balancer is fully completed and it is just for testing cli command, so it is may not Compatible with current balancer logic. could you also fix it in this PR?
|
|
||
| for (double utilization : nodeUtilizations) { | ||
| // select a random index from 0 to capacities.length | ||
| int index = ThreadLocalRandom.current().nextInt(0, capacities.length); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
createNodesInCluster seems not used
I'm initializing these maps here since the required initial size for these maps can change for every iteration. And resizing hash maps is a bit expensive. So should we prefer moving them to the constructor or we can leave them here for the garbage collector? |
JacksonYao287
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks @siddhantsangwan for this work! LGTM +1
waiting for a clean CI
lokeshj1703
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@siddhantsangwan Thanks for updating the PR! The changes look good to me. +1.
We also need to stop balancer when SCM is stopped.
|
@siddhantsangwan Thanks for the contribution! @JacksonYao287 Thanks for the reviews! I have committed the PR to master branch. |
|
Thanks for the reviews!
Yes, I've created HDDS-5603 for this. |
* master: HDDS-5358. Incorrect cache entry invalidation causes intermittent failure in testGetS3SecretAndRevokeS3Secret (apache#2518) HDDS-5608. Fix wrong command in ugrade doc (apache#2524) HDDS-5000. Run CI checks selectively (apache#2479) HDDS-4929. Select target datanodes and containers to move for Container Balancer (apache#2441) HDDS-5283. getStorageSize cast to int can cause issue (apache#2303) HDDS-5449 Recon namespace summary 'du' information should return replicated size of a key (apache#2489) HDDS-5558. vUnit invocation unit() may produce NPE (apache#2513) HDDS-5531. For Link Buckets avoid showing metadata. (apache#2502) HDDS-5549. Add 1.1 to supported versions in security policy (apache#2519) HDDS-5555. remove pipeline manager v1 code (apache#2511) HDDS-5546.OM Service ID change causes OM startup failure. (apache#2512) HDDS-5360. DN failed to process all delete block commands in one heartbeat interval (apache#2420) HDDS-5021. dev-support Dockerfile is badly outdated (apache#2480)
* master: HDDS-5358. Incorrect cache entry invalidation causes intermittent failure in testGetS3SecretAndRevokeS3Secret (apache#2518) HDDS-5608. Fix wrong command in ugrade doc (apache#2524) HDDS-5000. Run CI checks selectively (apache#2479) HDDS-4929. Select target datanodes and containers to move for Container Balancer (apache#2441) HDDS-5283. getStorageSize cast to int can cause issue (apache#2303) HDDS-5449 Recon namespace summary 'du' information should return replicated size of a key (apache#2489) HDDS-5558. vUnit invocation unit() may produce NPE (apache#2513) HDDS-5531. For Link Buckets avoid showing metadata. (apache#2502) HDDS-5549. Add 1.1 to supported versions in security policy (apache#2519) HDDS-5555. remove pipeline manager v1 code (apache#2511) HDDS-5546.OM Service ID change causes OM startup failure. (apache#2512) HDDS-5360. DN failed to process all delete block commands in one heartbeat interval (apache#2420) HDDS-5021. dev-support Dockerfile is badly outdated (apache#2480)
What changes were proposed in this pull request?
After determining over and under utilized nodes, container matches source with target and identifies containers to move. A source is either over or within threshold utilized, while a target is under utilized. The grouping of a target datanode and container to move is being called
ContainerMoveSelection. The selection must satisfy selection criteria, such as:The selection criteria currently not satisfied is: Try to move containers that are not following Placement Policy.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/HDDS-4929
How was this patch tested?
Added Unit tests:
TestContainerBalancer