Skip to content

HDFS-17372. CommandProcessingThread#queue should use LinkedBlockingDeque to prevent high priority command blocked by low priority command #6530

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

Open
wants to merge 3 commits into
base: trunk
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
Expand All @@ -62,6 +62,7 @@
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.InvalidBlockReportLeaseException;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
Expand Down Expand Up @@ -740,7 +741,18 @@ private void offerService() throws Exception {
if (state == HAServiceState.ACTIVE) {
handleRollingUpgradeStatus(resp);
}
commandProcessingThread.enqueue(resp.getCommands());
DatanodeCommand[] cmds = resp.getCommands();
if (cmds != null && cmds.length != 0) {
int length = cmds.length;
for (int i = length - 1; i >= 0; i--) {
if (cmds[i] instanceof KeyUpdateCommand) {
commandProcessingThread.enqueueFirst(cmds[i]);
cmds[i] = null;
break;
}
}
commandProcessingThread.enqueue(cmds);
}
isSlownode = resp.getIsSlownode();
}
}
Expand Down Expand Up @@ -1391,7 +1403,7 @@ class CommandProcessingThread extends Thread {
CommandProcessingThread(BPServiceActor actor) {
super("Command processor");
this.actor = actor;
this.queue = new LinkedBlockingQueue<>();
this.queue = new LinkedBlockingDeque<>();
setDaemon(true);
}

Expand Down Expand Up @@ -1478,6 +1490,22 @@ void enqueue(DatanodeCommand cmd) throws InterruptedException {
dn.getMetrics().incrActorCmdQueueLength(1);
}

/**
* Enqueue DatanodeCommand to the head of queue.
* @param cmd
* @throws InterruptedException
*/
void enqueueFirst(DatanodeCommand cmd) throws InterruptedException {
if (cmd == null) {
return;
}
((LinkedBlockingDeque<Runnable>) queue).putFirst(
() -> processCommand(new DatanodeCommand[]{cmd}));

LOG.info("Enqueue command: {} to the head of queue", cmd);
dn.getMetrics().incrActorCmdQueueLength(1);
}

void enqueue(List<DatanodeCommand> cmds) throws InterruptedException {
if (cmds == null) {
return;
Expand Down