fapifta commented on a change in pull request #881:
URL: https://github.com/apache/hadoop-ozone/pull/881#discussion_r420512977
##########
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:
Right, I see your point... I am not sure, haven't checked, but I guess
based on NetworkTopology, and maybe NodeManager we can tell how many healthy
racks we have, and if we have one, and the fallback is enabled, then we can
report the number of required replicas to be zero in the case we got down to 1
live rack only, while if fallback is not enabled, then we should report 1
additional replica to be required, and give a warning in the logs as the policy
can not be met.
I think it would be more clear to handle that in the validation of rack
aware policies if we can. What do you think?
Note that we are hardcoding fallback = true at the moment at the only place
where we call the policy factory in non-test code.
----------------------------------------------------------------
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]