|
@@ -18,18 +18,24 @@ package org.apache.hadoop.hdds.scm.container.replication;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Comparator;
|
|
|
+import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Objects;
|
|
|
import java.util.Set;
|
|
|
+import java.util.UUID;
|
|
|
import java.util.concurrent.ThreadFactory;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
|
|
+import org.apache.hadoop.hdds.scm.container.DeleteContainerCommandWatcher;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
|
|
.ContainerPlacementPolicy;
|
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
@@ -38,11 +44,14 @@ import org.apache.hadoop.hdds.server.events.EventQueue;
|
|
|
import org.apache.hadoop.hdds.server.events.IdentifiableEventPayload;
|
|
|
import org.apache.hadoop.ozone.lease.LeaseManager;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
|
|
|
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
+
|
|
|
+import static org.apache.hadoop.hdds.scm.events.SCMEvents
|
|
|
+ .TRACK_DELETE_CONTAINER_COMMAND;
|
|
|
import static org.apache.hadoop.hdds.scm.events.SCMEvents
|
|
|
.TRACK_REPLICATE_COMMAND;
|
|
|
import org.slf4j.Logger;
|
|
@@ -63,6 +72,7 @@ public class ReplicationManager implements Runnable {
|
|
|
private EventPublisher eventPublisher;
|
|
|
|
|
|
private ReplicationCommandWatcher replicationCommandWatcher;
|
|
|
+ private DeleteContainerCommandWatcher deleteContainerCommandWatcher;
|
|
|
|
|
|
private boolean running = true;
|
|
|
|
|
@@ -80,6 +90,11 @@ public class ReplicationManager implements Runnable {
|
|
|
new ReplicationCommandWatcher(TRACK_REPLICATE_COMMAND,
|
|
|
SCMEvents.REPLICATION_COMPLETE, commandWatcherLeaseManager);
|
|
|
|
|
|
+ this.deleteContainerCommandWatcher =
|
|
|
+ new DeleteContainerCommandWatcher(TRACK_DELETE_CONTAINER_COMMAND,
|
|
|
+ SCMEvents.DELETE_CONTAINER_COMMAND_COMPLETE,
|
|
|
+ commandWatcherLeaseManager);
|
|
|
+
|
|
|
this.replicationQueue = new ReplicationQueue();
|
|
|
|
|
|
eventQueue.addHandler(SCMEvents.REPLICATE_CONTAINER,
|
|
@@ -108,15 +123,15 @@ public class ReplicationManager implements Runnable {
|
|
|
request = replicationQueue.take();
|
|
|
|
|
|
ContainerID containerID = new ContainerID(request.getContainerId());
|
|
|
- ContainerInfo containerInfo =
|
|
|
- containerManager.getContainer(containerID);
|
|
|
-
|
|
|
- Preconditions.checkNotNull(containerInfo,
|
|
|
- "No information about the container " + request.getContainerId());
|
|
|
-
|
|
|
- Preconditions
|
|
|
- .checkState(containerInfo.getState() == LifeCycleState.CLOSED,
|
|
|
- "Container should be in closed state");
|
|
|
+ ContainerInfo container = containerManager.getContainer(containerID);
|
|
|
+ final HddsProtos.LifeCycleState state = container.getState();
|
|
|
+
|
|
|
+ if (state != LifeCycleState.CLOSED &&
|
|
|
+ state != LifeCycleState.QUASI_CLOSED) {
|
|
|
+ LOG.warn("Cannot replicate the container {} when in {} state.",
|
|
|
+ containerID, state);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
|
|
|
//check the current replication
|
|
|
List<ContainerReplica> containerReplicas =
|
|
@@ -130,28 +145,41 @@ public class ReplicationManager implements Runnable {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- ReplicationRequest finalRequest = request;
|
|
|
+ final ReplicationRequest finalRequest = request;
|
|
|
|
|
|
int inFlightReplications = replicationCommandWatcher.getTimeoutEvents(
|
|
|
- e -> e.request.getContainerId() == finalRequest.getContainerId())
|
|
|
+ e -> e.getRequest().getContainerId()
|
|
|
+ == finalRequest.getContainerId())
|
|
|
+ .size();
|
|
|
+
|
|
|
+ int inFlightDelete = deleteContainerCommandWatcher.getTimeoutEvents(
|
|
|
+ e -> e.getRequest().getContainerId()
|
|
|
+ == finalRequest.getContainerId())
|
|
|
.size();
|
|
|
|
|
|
int deficit =
|
|
|
- request.getExpecReplicationCount() - containerReplicas.size()
|
|
|
- - inFlightReplications;
|
|
|
+ (request.getExpecReplicationCount() - containerReplicas.size())
|
|
|
+ - (inFlightReplications - inFlightDelete);
|
|
|
|
|
|
if (deficit > 0) {
|
|
|
|
|
|
List<DatanodeDetails> datanodes = containerReplicas.stream()
|
|
|
+ .sorted((r1, r2) ->
|
|
|
+ r2.getSequenceId().compareTo(r1.getSequenceId()))
|
|
|
.map(ContainerReplica::getDatanodeDetails)
|
|
|
.collect(Collectors.toList());
|
|
|
List<DatanodeDetails> selectedDatanodes = containerPlacement
|
|
|
- .chooseDatanodes(datanodes, deficit,
|
|
|
- containerInfo.getUsedBytes());
|
|
|
+ .chooseDatanodes(datanodes, deficit, container.getUsedBytes());
|
|
|
|
|
|
//send the command
|
|
|
for (DatanodeDetails datanode : selectedDatanodes) {
|
|
|
|
|
|
+ LOG.info("Container {} is under replicated." +
|
|
|
+ " Expected replica count is {}, but found {}." +
|
|
|
+ " Re-replicating it on {}.",
|
|
|
+ container.containerID(), request.getExpecReplicationCount(),
|
|
|
+ containerReplicas.size(), datanode);
|
|
|
+
|
|
|
ReplicateContainerCommand replicateCommand =
|
|
|
new ReplicateContainerCommand(containerID.getId(), datanodes);
|
|
|
|
|
@@ -168,8 +196,62 @@ public class ReplicationManager implements Runnable {
|
|
|
}
|
|
|
|
|
|
} else if (deficit < 0) {
|
|
|
- //TODO: too many replicas. Not handled yet.
|
|
|
- LOG.debug("Too many replicas is not handled yet.");
|
|
|
+
|
|
|
+ int numberOfReplicasToDelete = Math.abs(deficit);
|
|
|
+
|
|
|
+ final Map<UUID, List<DatanodeDetails>> originIdToDnMap =
|
|
|
+ new LinkedHashMap<>();
|
|
|
+
|
|
|
+ containerReplicas.stream()
|
|
|
+ .sorted(Comparator.comparing(ContainerReplica::getSequenceId))
|
|
|
+ .forEach(replica -> {
|
|
|
+ originIdToDnMap.computeIfAbsent(
|
|
|
+ replica.getOriginDatanodeId(), key -> new ArrayList<>());
|
|
|
+ originIdToDnMap.get(replica.getOriginDatanodeId())
|
|
|
+ .add(replica.getDatanodeDetails());
|
|
|
+ });
|
|
|
+
|
|
|
+ for(UUID originId : originIdToDnMap.keySet()) {
|
|
|
+ final List<DatanodeDetails> listOfReplica =
|
|
|
+ originIdToDnMap.get(originId);
|
|
|
+ if (listOfReplica.size() > 1) {
|
|
|
+ final int toDelete = Math.min(listOfReplica.size() - 1,
|
|
|
+ numberOfReplicasToDelete);
|
|
|
+ final DeleteContainerCommand deleteContainer =
|
|
|
+ new DeleteContainerCommand(containerID.getId());
|
|
|
+ for (int i = 0; i < toDelete; i++) {
|
|
|
+ LOG.info("Container {} is over replicated." +
|
|
|
+ " Expected replica count is {}, but found {}." +
|
|
|
+ " Deleting the replica on {}.",
|
|
|
+ container.containerID(), request.getExpecReplicationCount(),
|
|
|
+ containerReplicas.size(), listOfReplica.get(i));
|
|
|
+ eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND,
|
|
|
+ new CommandForDatanode<>(listOfReplica.get(i).getUuid(),
|
|
|
+ deleteContainer));
|
|
|
+ DeletionRequestToRepeat timeoutEvent =
|
|
|
+ new DeletionRequestToRepeat(deleteContainer.getId(),
|
|
|
+ request);
|
|
|
+
|
|
|
+ eventPublisher.fireEvent(
|
|
|
+ TRACK_DELETE_CONTAINER_COMMAND, timeoutEvent);
|
|
|
+ }
|
|
|
+ numberOfReplicasToDelete -= toDelete;
|
|
|
+ }
|
|
|
+ if (numberOfReplicasToDelete == 0) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (numberOfReplicasToDelete != 0) {
|
|
|
+ final int expectedReplicaCount = container
|
|
|
+ .getReplicationFactor().getNumber();
|
|
|
+
|
|
|
+ LOG.warn("Not able to delete the container replica of Container" +
|
|
|
+ " {} even though it is over replicated. Expected replica" +
|
|
|
+ " count is {}, current replica count is {}.",
|
|
|
+ containerID, expectedReplicaCount,
|
|
|
+ expectedReplicaCount + numberOfReplicasToDelete);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
} catch (Exception e) {
|
|
@@ -196,17 +278,43 @@ public class ReplicationManager implements Runnable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Event for the ReplicationCommandWatcher to repeate the embedded request.
|
|
|
+ * Event for the ReplicationCommandWatcher to repeat the embedded request.
|
|
|
* in case fof timeout.
|
|
|
*/
|
|
|
public static class ReplicationRequestToRepeat
|
|
|
+ extends ContainerRequestToRepeat {
|
|
|
+
|
|
|
+ public ReplicationRequestToRepeat(
|
|
|
+ long commandId, ReplicationRequest request) {
|
|
|
+ super(commandId, request);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Event for the DeleteContainerCommandWatcher to repeat the
|
|
|
+ * embedded request. In case fof timeout.
|
|
|
+ */
|
|
|
+ public static class DeletionRequestToRepeat
|
|
|
+ extends ContainerRequestToRepeat {
|
|
|
+
|
|
|
+ public DeletionRequestToRepeat(
|
|
|
+ long commandId, ReplicationRequest request) {
|
|
|
+ super(commandId, request);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Container Request wrapper which will be used by ReplicationManager to
|
|
|
+ * perform the intended operation.
|
|
|
+ */
|
|
|
+ public static class ContainerRequestToRepeat
|
|
|
implements IdentifiableEventPayload {
|
|
|
|
|
|
private final long commandId;
|
|
|
|
|
|
private final ReplicationRequest request;
|
|
|
|
|
|
- public ReplicationRequestToRepeat(long commandId,
|
|
|
+ ContainerRequestToRepeat(long commandId,
|
|
|
ReplicationRequest request) {
|
|
|
this.commandId = commandId;
|
|
|
this.request = request;
|
|
@@ -229,7 +337,7 @@ public class ReplicationManager implements Runnable {
|
|
|
if (o == null || getClass() != o.getClass()) {
|
|
|
return false;
|
|
|
}
|
|
|
- ReplicationRequestToRepeat that = (ReplicationRequestToRepeat) o;
|
|
|
+ ContainerRequestToRepeat that = (ContainerRequestToRepeat) o;
|
|
|
return Objects.equals(request, that.request);
|
|
|
}
|
|
|
|
|
@@ -241,7 +349,7 @@ public class ReplicationManager implements Runnable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Add javadoc.
|
|
|
+ * Event which indicates that the replicate operation is completed.
|
|
|
*/
|
|
|
public static class ReplicationCompleted
|
|
|
implements IdentifiableEventPayload {
|
|
@@ -257,4 +365,22 @@ public class ReplicationManager implements Runnable {
|
|
|
return uuid;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Event which indicates that the container deletion operation is completed.
|
|
|
+ */
|
|
|
+ public static class DeleteContainerCommandCompleted
|
|
|
+ implements IdentifiableEventPayload {
|
|
|
+
|
|
|
+ private final long uuid;
|
|
|
+
|
|
|
+ public DeleteContainerCommandCompleted(long uuid) {
|
|
|
+ this.uuid = uuid;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getId() {
|
|
|
+ return uuid;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|