sodonnel commented on a change in pull request #881:
URL: https://github.com/apache/hadoop-ozone/pull/881#discussion_r420203008
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -512,25 +523,60 @@ private void handleUnderReplicatedContainer(final
ContainerInfo container,
if (source.size() > 0) {
final int replicationFactor = container
.getReplicationFactor().getNumber();
- final int delta = replicationFactor - getReplicaCount(id, replicas);
+ // Want to check if the container is mis-replicated after considering
+ // inflight add and delete.
+ // Create a new list from source (healthy replicas minus pending
delete)
+ List<DatanodeDetails> targetReplicas = new ArrayList<>(source);
+ // Then add any pending additions
+ targetReplicas.addAll(replicationInFlight);
+
+ int delta = replicationFactor - getReplicaCount(id, replicas);
+ final ContainerPlacementStatus placementStatus =
+ containerPlacement.validateContainerPlacement(
+ targetReplicas, replicationFactor);
+ final int misRepDelta = placementStatus.additionalReplicaRequired();
+ final int replicasNeeded
+ = delta < misRepDelta ? misRepDelta : delta;
+
final List<DatanodeDetails> excludeList = replicas.stream()
.map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList());
+ excludeList.addAll(replicationInFlight);
List<InflightAction> actionList = inflightReplication.get(id);
if (actionList != null) {
actionList.stream().map(r -> r.datanode)
.forEach(excludeList::add);
}
final List<DatanodeDetails> selectedDatanodes = containerPlacement
- .chooseDatanodes(excludeList, null, delta,
+ .chooseDatanodes(excludeList, null, replicasNeeded,
container.getUsedBytes());
-
- LOG.info("Container {} is under replicated. Expected replica count" +
- " is {}, but found {}.", id, replicationFactor,
- replicationFactor - delta);
-
- for (DatanodeDetails datanode : selectedDatanodes) {
- sendReplicateCommand(container, datanode, source);
+ if (delta > 0) {
+ LOG.info("Container {} is under replicated. Expected replica count" +
+ " is {}, but found {}.", id, replicationFactor,
+ replicationFactor - delta);
+ }
+ int newMisRepDelta = misRepDelta;
+ if (misRepDelta > 0) {
+ LOG.info("Container: {}. {}",
+ id, placementStatus.misReplicatedReason());
+ // Check if the new target nodes (original plus newly selected nodes)
+ // makes the placement policy valid.
+ targetReplicas.addAll(selectedDatanodes);
+ newMisRepDelta = containerPlacement.validateContainerPlacement(
+ targetReplicas, replicationFactor).additionalReplicaRequired();
Review comment:
> The rack-aware policy's chooseDatanodes() method we use ensures that
based on the excluded nodes list we get back a proper list
This is where it gets tricky. There is a fallback (certainly in the pipeline
placement policy, but I have not checked the ContainerPlacementRackAware in
complete detail) where if the placement policy cannot select enough racks, it
will give back nodes which do not meet the placement.
This is to ensure (in the case of pipelinePlacementPolicy) that the cluster
can keep on operating if only 1 rack is available for some reason. For the
closed container placement, this should be the case too, as if you are down to
1 replica, it is better it gets replicated onto more nodes on the same rack,
rather than risk losing the data.
We probably need a future change to scrub the bad pipelines if this fall
back happens, while replication manager should deal with closed containers.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]