-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-8153. Integrate ContainerBalancer with MoveManager #4391
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
Merged
Merged
Changes from 2 commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
293da24
HDDS-8153. Integrate ContainerBalancer with MoveManager
siddhantsangwan 68d7c1e
Merge branch 'master' into HDDS-8153
siddhantsangwan ba5c810
change config defaults and address review
siddhantsangwan b90a708
revert changes related to timeouts. Reset moveManager's state before …
siddhantsangwan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,7 @@ | |
| import org.apache.hadoop.hdds.scm.container.ContainerInfo; | ||
| import org.apache.hadoop.hdds.scm.container.ContainerManager; | ||
| import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; | ||
| import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException; | ||
| import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; | ||
| import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; | ||
| import org.apache.hadoop.hdds.scm.ha.SCMContext; | ||
|
|
@@ -44,6 +45,7 @@ | |
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.NavigableSet; | ||
|
|
@@ -68,6 +70,7 @@ public class ContainerBalancerTask implements Runnable { | |
| private NodeManager nodeManager; | ||
| private ContainerManager containerManager; | ||
| private ReplicationManager replicationManager; | ||
| private MoveManager moveManager; | ||
| private OzoneConfiguration ozoneConfiguration; | ||
| private ContainerBalancer containerBalancer; | ||
| private final SCMContext scmContext; | ||
|
|
@@ -131,6 +134,7 @@ public ContainerBalancerTask(StorageContainerManager scm, | |
| this.nodeManager = scm.getScmNodeManager(); | ||
| this.containerManager = scm.getContainerManager(); | ||
| this.replicationManager = scm.getReplicationManager(); | ||
| this.moveManager = new MoveManager(replicationManager, containerManager); | ||
| this.ozoneConfiguration = scm.getConfiguration(); | ||
| this.containerBalancer = containerBalancer; | ||
| this.config = config; | ||
|
|
@@ -592,7 +596,7 @@ private void checkIterationMoveResults() { | |
| LOG.warn("Container balancer is interrupted"); | ||
| Thread.currentThread().interrupt(); | ||
| } catch (TimeoutException e) { | ||
| long timeoutCounts = cancelAndCountPendingMoves(); | ||
| long timeoutCounts = cancelMovesThatExceedTimeoutDuration(); | ||
| LOG.warn("{} Container moves are canceled.", timeoutCounts); | ||
| metrics.incrementNumContainerMovesTimeoutInLatestIteration(timeoutCounts); | ||
| } catch (ExecutionException e) { | ||
|
|
@@ -621,18 +625,42 @@ private void checkIterationMoveResults() { | |
| metrics.getNumContainerMovesCompletedInLatestIteration()); | ||
| } | ||
|
|
||
| private long cancelAndCountPendingMoves() { | ||
| return moveSelectionToFutureMap.entrySet().stream() | ||
| .filter(entry -> !entry.getValue().isDone()) | ||
| .peek(entry -> { | ||
| LOG.warn("Container move timeout for container {} from source {}" + | ||
| " to target {}.", | ||
| entry.getKey().getContainerID(), | ||
| containerToSourceMap.get(entry.getKey().getContainerID()) | ||
| .getUuidString(), | ||
| entry.getKey().getTargetNode().getUuidString()); | ||
| entry.getValue().cancel(true); | ||
| }).count(); | ||
| /** | ||
| * Cancels container moves that are not yet done. Note that if a move | ||
| * command has already been sent out to a Datanode, we don't yet have the | ||
| * capability to cancel it. However, those commands in the DN should time out | ||
| * if they haven't been processed yet. | ||
| * | ||
| * @return number of moves that did not complete (timed out) and were | ||
| * cancelled. | ||
| */ | ||
| private long cancelMovesThatExceedTimeoutDuration() { | ||
| Set<Map.Entry<ContainerMoveSelection, | ||
| CompletableFuture<MoveManager.MoveResult>>> | ||
| entries = moveSelectionToFutureMap.entrySet(); | ||
| Iterator<Map.Entry<ContainerMoveSelection, | ||
| CompletableFuture<MoveManager.MoveResult>>> | ||
| iterator = entries.iterator(); | ||
|
|
||
| int numCancelled = 0; | ||
| // iterate through all moves and cancel ones that aren't done yet | ||
| while (iterator.hasNext()) { | ||
| Map.Entry<ContainerMoveSelection, | ||
| CompletableFuture<MoveManager.MoveResult>> | ||
| entry = iterator.next(); | ||
| if (!entry.getValue().isDone()) { | ||
| LOG.warn("Container move timed out for container {} from source {}" + | ||
| " to target {}.", entry.getKey().getContainerID(), | ||
| containerToSourceMap.get(entry.getKey().getContainerID()) | ||
| .getUuidString(), | ||
| entry.getKey().getTargetNode().getUuidString()); | ||
|
|
||
| entry.getValue().cancel(true); | ||
| numCancelled += 1; | ||
| } | ||
| } | ||
|
|
||
| return numCancelled; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -742,13 +770,13 @@ private boolean adaptOnReachingIterationLimits() { | |
| } | ||
|
|
||
| /** | ||
| * Asks {@link ReplicationManager} to move the specified container from | ||
| * source to target. | ||
| * Asks {@link ReplicationManager} or {@link MoveManager} to move the | ||
| * specified container from source to target. | ||
| * | ||
| * @param source the source datanode | ||
| * @param moveSelection the selected container to move and target datanode | ||
| * @return false if an exception occurred or the move completed with a | ||
| * result other than ReplicationManager.MoveResult.COMPLETED. Returns true | ||
| * result other than MoveManager.MoveResult.COMPLETED. Returns true | ||
| * if the move completed with MoveResult.COMPLETED or move is not yet done | ||
| */ | ||
| private boolean moveContainer(DatanodeDetails source, | ||
|
|
@@ -757,48 +785,66 @@ private boolean moveContainer(DatanodeDetails source, | |
| CompletableFuture<MoveManager.MoveResult> future; | ||
| try { | ||
| ContainerInfo containerInfo = containerManager.getContainer(containerID); | ||
| future = replicationManager | ||
| .move(containerID, source, moveSelection.getTargetNode()) | ||
| .whenComplete((result, ex) -> { | ||
|
|
||
| metrics.incrementCurrentIterationContainerMoveMetric(result, 1); | ||
| if (ex != null) { | ||
| LOG.info("Container move for container {} from source {} to " + | ||
| "target {} failed with exceptions {}", | ||
| containerID.toString(), | ||
|
|
||
| ReplicationManager.ReplicationManagerConfiguration rmConf = | ||
| ozoneConfiguration.getObject( | ||
| ReplicationManager.ReplicationManagerConfiguration.class); | ||
| /* | ||
| If LegacyReplicationManager is enabled, ReplicationManager will | ||
| redirect to it. Otherwise, use MoveManager. | ||
| */ | ||
| if (rmConf.isLegacyEnabled()) { | ||
| future = replicationManager | ||
| .move(containerID, source, moveSelection.getTargetNode()); | ||
| } else { | ||
| future = moveManager.move(containerID, source, | ||
| moveSelection.getTargetNode()); | ||
| } | ||
|
|
||
| future = future.whenComplete((result, ex) -> { | ||
| metrics.incrementCurrentIterationContainerMoveMetric(result, 1); | ||
| if (ex != null) { | ||
| LOG.info("Container move for container {} from source {} to " + | ||
| "target {} failed with exceptions.", | ||
| containerID.toString(), | ||
| source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString(), ex); | ||
| metrics.incrementNumContainerMovesFailedInLatestIteration(1); | ||
| } else { | ||
| if (result == MoveManager.MoveResult.COMPLETED) { | ||
| sizeActuallyMovedInLatestIteration += | ||
| containerInfo.getUsedBytes(); | ||
| if (LOG.isDebugEnabled()) { | ||
|
||
| LOG.debug("Container move completed for container {} from " + | ||
| "source {} to target {}", containerID, | ||
| source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString(), ex); | ||
| metrics.incrementNumContainerMovesFailedInLatestIteration(1); | ||
| } else { | ||
| if (result == MoveManager.MoveResult.COMPLETED) { | ||
| sizeActuallyMovedInLatestIteration += | ||
| containerInfo.getUsedBytes(); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Container move completed for container {} from " + | ||
| "source {} to target {}", containerID, | ||
| source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString()); | ||
| } | ||
| } else { | ||
| LOG.warn( | ||
| "Container move for container {} from source {} to target" + | ||
| " {} failed: {}", | ||
| moveSelection.getContainerID(), source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString(), result); | ||
| } | ||
| moveSelection.getTargetNode().getUuidString()); | ||
| } | ||
| }); | ||
| } else { | ||
| LOG.warn( | ||
| "Container move for container {} from source {} to target" + | ||
| " {} failed: {}", | ||
| moveSelection.getContainerID(), source.getUuidString(), | ||
| moveSelection.getTargetNode().getUuidString(), result); | ||
| } | ||
| } | ||
| }); | ||
| } catch (ContainerNotFoundException e) { | ||
| LOG.warn("Could not find Container {} for container move", | ||
| containerID, e); | ||
| metrics.incrementNumContainerMovesFailedInLatestIteration(1); | ||
| return false; | ||
| } catch (NodeNotFoundException | TimeoutException e) { | ||
| } catch (NodeNotFoundException | TimeoutException | | ||
| ContainerReplicaNotFoundException e) { | ||
| LOG.warn("Container move failed for container {}", containerID, e); | ||
| metrics.incrementNumContainerMovesFailedInLatestIteration(1); | ||
| return false; | ||
| } | ||
|
|
||
| /* | ||
| If the future hasn't failed yet, put it in moveSelectionToFutureMap for | ||
| processing later | ||
| */ | ||
| if (future.isDone()) { | ||
| if (future.isCompletedExceptionally()) { | ||
| return false; | ||
|
|
@@ -1045,6 +1091,16 @@ void setConfig(ContainerBalancerConfiguration config) { | |
| this.config = config; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void setMoveManager(MoveManager moveManager) { | ||
| this.moveManager = moveManager; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void setTaskStatus(Status taskStatus) { | ||
| this.taskStatus = taskStatus; | ||
| } | ||
|
|
||
| public Status getBalancerStatus() { | ||
| return taskStatus; | ||
| } | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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: I think this should be stored in the constructor.