JacksonYao287 commented on a change in pull request #2349:
URL: https://github.com/apache/ozone/pull/2349#discussion_r660238109
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -862,16 +1061,55 @@ private void handleOverReplicatedContainer(final
ContainerInfo container,
break;
}
}
+
+ eligibleReplicas.removeAll(unhealthyReplicas);
+ boolean isInMove = inflightMove.containsKey(id);
+ boolean isSourceDnInReplicaSet = false;
+ boolean isTargetDnInReplicaSet = false;
+
+ if (isInMove) {
+ Pair<DatanodeDetails, DatanodeDetails> movePair =
+ inflightMove.get(id);
+ final DatanodeDetails sourceDN = movePair.getKey();
+ isSourceDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails().equals(sourceDN));
+ isTargetDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails()
+ .equals(movePair.getValue()));
+ int sourceDnPos = 0;
+ for (int i = 0; i < eligibleReplicas.size(); i++) {
+ if (eligibleReplicas.get(i).getDatanodeDetails()
+ .equals(sourceDN)) {
+ sourceDnPos = i;
+ break;
+ }
+ }
Review comment:
i get your point here, but sourceDN is a type of DatanodeDetails and
eligibleReplicas is a list of type ContainerReplica, so i am not sure
`eligibleReplicas.remove(sourceDN)` can work as expected
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -862,16 +1061,55 @@ private void handleOverReplicatedContainer(final
ContainerInfo container,
break;
}
}
+
+ eligibleReplicas.removeAll(unhealthyReplicas);
+ boolean isInMove = inflightMove.containsKey(id);
+ boolean isSourceDnInReplicaSet = false;
+ boolean isTargetDnInReplicaSet = false;
+
+ if (isInMove) {
+ Pair<DatanodeDetails, DatanodeDetails> movePair =
+ inflightMove.get(id);
+ final DatanodeDetails sourceDN = movePair.getKey();
+ isSourceDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails().equals(sourceDN));
+ isTargetDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails()
+ .equals(movePair.getValue()));
+ int sourceDnPos = 0;
+ for (int i = 0; i < eligibleReplicas.size(); i++) {
+ if (eligibleReplicas.get(i).getDatanodeDetails()
+ .equals(sourceDN)) {
+ sourceDnPos = i;
+ break;
+ }
+ }
+ if (isTargetDnInReplicaSet) {
+ // if the container is in inflightMove and target datanode is
+ // included in the replicas, then swap the source datanode to
+ // first of the replica list if exists, so the source datanode
+ // will be first removed if possible.
+ eligibleReplicas.add(0, eligibleReplicas.remove(sourceDnPos));
+ } else {
+ // a container replica that being moved should not be removed.
+ // if the container is in inflightMove and target datanode is not
+ // included in the replicas, then swap the source datanode to the
+ // last of the replica list, so the source datanode will not
+ // be removed.
+ eligibleReplicas.add(eligibleReplicas.remove(sourceDnPos));
Review comment:
> TargetDN would receive all the existing replicas as source to copy from
in move option, we maintain src and target in inflightMove and track the
replication from source to target. if we take other the existing replicas as
the source, Although the final result seems same(sourceDN is not present and
targetDN is present), it seems a little confused in Semantics. i think it will
better off making sure the source not be deleted until the target is present.
> I think that at end if sourceDN is not present and targetDN is present
move is successful.
yes , it make sense. if the target is present, and source disappear somehow,
we can consider move is successful.
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -862,16 +1061,55 @@ private void handleOverReplicatedContainer(final
ContainerInfo container,
break;
}
}
+
+ eligibleReplicas.removeAll(unhealthyReplicas);
+ boolean isInMove = inflightMove.containsKey(id);
+ boolean isSourceDnInReplicaSet = false;
+ boolean isTargetDnInReplicaSet = false;
+
+ if (isInMove) {
+ Pair<DatanodeDetails, DatanodeDetails> movePair =
+ inflightMove.get(id);
+ final DatanodeDetails sourceDN = movePair.getKey();
+ isSourceDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails().equals(sourceDN));
+ isTargetDnInReplicaSet = eligibleReplicas.stream()
+ .anyMatch(r -> r.getDatanodeDetails()
+ .equals(movePair.getValue()));
+ int sourceDnPos = 0;
+ for (int i = 0; i < eligibleReplicas.size(); i++) {
+ if (eligibleReplicas.get(i).getDatanodeDetails()
+ .equals(sourceDN)) {
+ sourceDnPos = i;
+ break;
+ }
+ }
+ if (isTargetDnInReplicaSet) {
+ // if the container is in inflightMove and target datanode is
+ // included in the replicas, then swap the source datanode to
+ // first of the replica list if exists, so the source datanode
+ // will be first removed if possible.
+ eligibleReplicas.add(0, eligibleReplicas.remove(sourceDnPos));
Review comment:
yes, will handle this
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -471,6 +549,127 @@ private void updateInflightAction(final ContainerInfo
container,
}
}
+ /**
+ * add a move action for a given container.
+ *
+ * @param cid Container to move
+ * @param srcDn datanode to move from
+ * @param targetDn datanode to move to
+ */
+ public Optional<CompletableFuture<MoveResult>> move(ContainerID cid,
+ DatanodeDetails srcDn, DatanodeDetails targetDn)
+ throws ContainerNotFoundException, NodeNotFoundException {
+ LOG.info("receive a move requset about container {} , from {} to {}",
+ cid, srcDn.getUuid(), targetDn.getUuid());
+ Optional<CompletableFuture<MoveResult>> ret = Optional.empty();
+ if (!isRunning()) {
+ LOG.info("Replication Manager in not running. please start it first");
+ return ret;
+ }
+
+ /*
+ * make sure the flowing conditions are met:
+ * 1 the given two datanodes are in healthy state
+ * 2 the given container exists on the given source datanode
+ * 3 the given container does not exist on the given target datanode
+ * 4 the given container is in closed state
+ * 5 the giver container is not taking any inflight action
+ * 6 the given two datanodes are in IN_SERVICE state
+ *
+ * move is a combination of two steps : replication and deletion.
+ * if the conditions above are all met, then we take a conservative
+ * strategy here : replication can always be executed, but the execution
+ * of deletion always depends on placement policy
+ */
+
+ NodeStatus currentNodeStat = nodeManager.getNodeStatus(srcDn);
+ NodeState healthStat = currentNodeStat.getHealth();
+ NodeOperationalState operationalState =
+ currentNodeStat.getOperationalState();
+ if (healthStat != NodeState.HEALTHY) {
+ LOG.info("given source datanode is in {} state, " +
+ "not in HEALTHY state", healthStat);
+ return ret;
+ }
+ if (operationalState != NodeOperationalState.IN_SERVICE) {
+ LOG.info("given source datanode is in {} state, " +
+ "not in IN_SERVICE state", operationalState);
+ return ret;
+ }
+
+ currentNodeStat = nodeManager.getNodeStatus(targetDn);
+ healthStat = currentNodeStat.getHealth();
+ operationalState = currentNodeStat.getOperationalState();
+ if (healthStat != NodeState.HEALTHY) {
+ LOG.info("given target datanode is in {} state, " +
+ "not in HEALTHY state", healthStat);
+ return ret;
+ }
+ if (operationalState != NodeOperationalState.IN_SERVICE) {
+ LOG.info("given target datanode is in {} state, " +
+ "not in IN_SERVICE state", operationalState);
+ return ret;
+ }
+
+ // we need to synchronize on ContainerInfo, since it is
+ // shared by ICR/FCR handler and this.processContainer
+ // TODO: use a Read lock after introducing a RW lock into ContainerInfo
+ ContainerInfo cif = containerManager.getContainer(cid);
+ synchronized (cif) {
+ final Set<DatanodeDetails> replicas = containerManager
+ .getContainerReplicas(cid).stream()
+ .map(ContainerReplica::getDatanodeDetails)
+ .collect(Collectors.toSet());
+ if (replicas.contains(targetDn)) {
+ LOG.info("given container exists in the target Datanode");
+ return ret;
+ }
+ if (!replicas.contains(srcDn)) {
+ LOG.info("given container does not exist in the source Datanode");
+ return ret;
+ }
+
+ /*
+ * the reason why the given container should not be taking any inflight
+ * action is that: if the given container is being replicated or deleted,
+ * the num of its replica is not deterministic, so move operation issued
+ * by balancer may cause a nondeterministic result, so we should drop
+ * this option for this time.
+ * */
+
+ if (inflightReplication.containsKey(cid)) {
+ LOG.info("given container is in inflight replication");
+ return ret;
+ }
+ if (inflightDeletion.containsKey(cid)) {
+ LOG.info("given container is in inflight deletion");
+ return ret;
+ }
+
+ /*
+ * here, no need to see whether cid is in inflightMove, because
+ * these three map are all synchronized on ContainerInfo, if cid
+ * is in infligtMove , it must now being replicated or deleted,
+ * so it must be in inflightReplication or in infligthDeletion.
+ * thus, if we can not find cid in both of them , this cid must
+ * not be in inflightMove.
+ */
+
Review comment:
to be simple, I take a conservative strategy here : replication can
always be executed, but the execution of deletion always depends on placement
policy, i think this do no harm to data safety. the placement policy check is
delayed to `handleOverReplicatedContainer`
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -471,6 +549,127 @@ private void updateInflightAction(final ContainerInfo
container,
}
}
+ /**
+ * add a move action for a given container.
+ *
+ * @param cid Container to move
+ * @param srcDn datanode to move from
+ * @param targetDn datanode to move to
+ */
+ public Optional<CompletableFuture<MoveResult>> move(ContainerID cid,
Review comment:
it seem good, i think completed exceptionally is a better choice. i will
do this
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
##########
@@ -471,6 +549,127 @@ private void updateInflightAction(final ContainerInfo
container,
}
}
+ /**
+ * add a move action for a given container.
+ *
+ * @param cid Container to move
+ * @param srcDn datanode to move from
+ * @param targetDn datanode to move to
+ */
+ public Optional<CompletableFuture<MoveResult>> move(ContainerID cid,
Review comment:
it seem good, i will add more MoveResult for the errors
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]