-
Notifications
You must be signed in to change notification settings - Fork 594
HDDS-6848. Ignore timeout replication tasks on datanode #3497
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,8 @@ | |
| */ | ||
| package org.apache.hadoop.ozone.container.replication; | ||
|
|
||
| import java.time.Duration; | ||
| import java.time.Instant; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentHashMap.KeySetView; | ||
| import java.util.concurrent.ExecutorService; | ||
|
|
@@ -53,6 +55,7 @@ public class ReplicationSupervisor { | |
| private final AtomicLong requestCounter = new AtomicLong(); | ||
| private final AtomicLong successCounter = new AtomicLong(); | ||
| private final AtomicLong failureCounter = new AtomicLong(); | ||
| private final AtomicLong timeoutCounter = new AtomicLong(); | ||
|
|
||
| /** | ||
| * A set of container IDs that are currently being downloaded | ||
|
|
@@ -147,6 +150,17 @@ public void run() { | |
| final Long containerId = task.getContainerId(); | ||
| try { | ||
| requestCounter.incrementAndGet(); | ||
| if (task.getTimeoutMs() != 0) { | ||
| long msInQueue = | ||
| Duration.between(task.getQueued(), Instant.now()).toMillis(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There are 2 queues in the DN, the main command queue and then the replication queue. The time set for getQueued() is the time the command is placed onto the replication queue. However, there are some delays in the system here eg:
I think this would mean the pending operation in SCM could expire before DN command does, and it gives SCM a chance to schedule another command before this gets expired on the DN. I think it might be better if we set the expiry time on the command (in ms since epoch) when it is created in SCM to match the SCM timeout, and then it avoids any of these delays. Infact, it may make sense to make the DN command expiry a little less than the SCM timeout, so the DN command tends to expire slightly earlier than the SCM timeout. |
||
| if (msInQueue > task.getTimeoutMs()) { | ||
| LOG.info("Ignore this replicate container command for container" + | ||
| " {} since queueTime larger then timeout {}ms", | ||
| containerId, task.getTimeoutMs()); | ||
| timeoutCounter.incrementAndGet(); | ||
| return; | ||
| } | ||
| } | ||
|
|
||
| if (context != null) { | ||
| DatanodeDetails dn = context.getParent().getDatanodeDetails(); | ||
|
|
@@ -206,4 +220,8 @@ public long getReplicationSuccessCount() { | |
| public long getReplicationFailureCount() { | ||
| return failureCounter.get(); | ||
| } | ||
|
|
||
| public long getReplicationTimeoutCount() { | ||
| return timeoutCounter.get(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,20 +41,28 @@ public class ReplicateContainerCommand | |
|
|
||
| private final long containerID; | ||
| private final List<DatanodeDetails> sourceDatanodes; | ||
| private final long timeoutMs; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we go with my earlier suggestion of setting the "run by time" in SCM on the command, perhaps rename "timeout" to something like "deadlineEpochMs" or "expiryEpochMs" throughout this method. |
||
|
|
||
| public ReplicateContainerCommand(long containerID, | ||
| List<DatanodeDetails> sourceDatanodes) { | ||
| this(containerID, sourceDatanodes, 0L); | ||
| } | ||
|
|
||
| public ReplicateContainerCommand(long containerID, | ||
| List<DatanodeDetails> sourceDatanodes, long timeoutMs) { | ||
| super(); | ||
| this.containerID = containerID; | ||
| this.sourceDatanodes = sourceDatanodes; | ||
| this.timeoutMs = timeoutMs; | ||
| } | ||
|
|
||
| // Should be called only for protobuf conversion | ||
| public ReplicateContainerCommand(long containerID, | ||
| List<DatanodeDetails> sourceDatanodes, long id) { | ||
| List<DatanodeDetails> sourceDatanodes, long id, long timeoutMs) { | ||
| super(id); | ||
| this.containerID = containerID; | ||
| this.sourceDatanodes = sourceDatanodes; | ||
| this.timeoutMs = timeoutMs; | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -66,7 +74,8 @@ public Type getType() { | |
| public ReplicateContainerCommandProto getProto() { | ||
| Builder builder = ReplicateContainerCommandProto.newBuilder() | ||
| .setCmdId(getId()) | ||
| .setContainerID(containerID); | ||
| .setContainerID(containerID) | ||
| .setTimeoutMs(timeoutMs); | ||
| for (DatanodeDetails dd : sourceDatanodes) { | ||
| builder.addSources(dd.getProtoBufMessage()); | ||
| } | ||
|
|
@@ -84,7 +93,7 @@ public static ReplicateContainerCommand getFromProtobuf( | |
| .collect(Collectors.toList()); | ||
|
|
||
| return new ReplicateContainerCommand(protoMessage.getContainerID(), | ||
| datanodeDetails, protoMessage.getCmdId()); | ||
| datanodeDetails, protoMessage.getCmdId(), protoMessage.getTimeoutMs()); | ||
|
|
||
| } | ||
|
|
||
|
|
@@ -95,4 +104,8 @@ public long getContainerID() { | |
| public List<DatanodeDetails> getSourceDatanodes() { | ||
| return sourceDatanodes; | ||
| } | ||
|
|
||
| public long getTimeoutMs() { | ||
| return timeoutMs; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -227,6 +227,34 @@ public void slowDownload() { | |
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testTimeoutTask() { | ||
| // GIVEN | ||
| ReplicationSupervisor supervisor = supervisorWith(__ -> slowReplicator, | ||
| new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, | ||
| new LinkedBlockingQueue<>())); | ||
|
|
||
| try { | ||
| //WHEN | ||
| supervisor.addTask(new ReplicationTask(1L, emptyList())); | ||
| supervisor.addTask(new ReplicationTask(2L, emptyList(), 100L)); | ||
| supervisor.addTask(new ReplicationTask(3L, emptyList(), 100L)); | ||
|
|
||
| //THEN | ||
| Assert.assertEquals(3, supervisor.getInFlightReplications()); | ||
| Assert.assertEquals(2, supervisor.getQueueSize()); | ||
| // Sleep 2s, wait all tasks processed | ||
| try { | ||
| Thread.sleep(2000); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we inject the MontonicClock into the Supervisor, then we can create it with a instance of |
||
| } catch (InterruptedException e) { | ||
| } | ||
| Assert.assertEquals(0, supervisor.getInFlightReplications()); | ||
| Assert.assertEquals(0, supervisor.getQueueSize()); | ||
| Assert.assertEquals(2, supervisor.getReplicationTimeoutCount()); | ||
| } finally { | ||
| supervisor.stop(); | ||
| } | ||
| } | ||
| @Test | ||
| public void testDownloadAndImportReplicatorFailure() { | ||
| ReplicationSupervisor supervisor = | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -409,6 +409,7 @@ message ReplicateContainerCommandProto { | |
| required int64 containerID = 1; | ||
| repeated DatanodeDetailsProto sources = 2; | ||
| required int64 cmdId = 3; | ||
| optional int64 timeoutMs = 4; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we rename the timeout to expiry or deadline, then change it here too. |
||
| } | ||
|
|
||
| /** | ||
|
|
||
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.
Rather than calling Instance.now here, can we inject an instance of
MonotonicClockand use it to get the time. This will make tests easier and avoidsleepcalls in the tests. This is something we have used in other places in the code before.