Skip to content

HDFS-15869. Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang #2737

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 9 commits into
base: trunk
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -396,6 +396,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
"dfs.namenode.edits.asynclogging.pending.queue.size";
public static final int
DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT = 4096;
public static final String
DFS_NAMENODE_EDITS_ASYNC_LOGSYNCNOTIFY_EXECUTOR_SIZE =
"dfs.namenode.edits.async.logsyncnotify.executor.size";
public static final int
DFS_NAMENODE_EDITS_ASYNC_LOGSYNCNOTIFY_EXECUTOR_SIZE_DEFAULT = 10;

public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
Expand All @@ -50,6 +52,14 @@ class FSEditLogAsync extends FSEditLog implements Runnable {
// requires concurrent access from caller threads and syncing thread.
private final BlockingQueue<Edit> editPendingQ;

// Thread pool for executing logSyncNotify
// It should not be shut down when the FSEditLogAsync is closed, because
// 1) when the state transitions from active to standby and then transitions back,
// it is still using the same FSEditLogAsync object, so the same executor should
// keep working; 2) in all possible scenarios, FSEditLogAsync is a singleton
// in the NameNode process, so it is fine to let this executor run forever.
private final ExecutorService logSyncNotifyExecutor;

// only accessed by syncing thread so no synchronization required.
// queue is unbounded because it's effectively limited by the size
// of the edit log buffer - ie. a sync will eventually be forced.
Expand All @@ -67,6 +77,12 @@ class FSEditLogAsync extends FSEditLog implements Runnable {
DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT);

editPendingQ = new ArrayBlockingQueue<>(editPendingQSize);

int logSyncNotifyExecutorSize = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGSYNCNOTIFY_EXECUTOR_SIZE,
DFSConfigKeys.
DFS_NAMENODE_EDITS_ASYNC_LOGSYNCNOTIFY_EXECUTOR_SIZE_DEFAULT);
logSyncNotifyExecutor = Executors.newFixedThreadPool(logSyncNotifyExecutorSize);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I still think that we would be better to shutdown logSyncNotifyExecutor in close method and re-initialize logSyncNotifyExecutor in FSEditLogAsync#openForWrite function. Every time NN does the failover, it will invoke startActiveServices method and FSEditLogAsync#openForWrite will be executed. Related code: FSNamesystem.java#L1373

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @linyiqun for your comments. It makes sense to me.

}

private boolean isSyncThreadAlive() {
Expand Down Expand Up @@ -263,7 +279,9 @@ public void run() {
syncEx = ex;
}
while ((edit = syncWaitQ.poll()) != null) {
edit.logSyncNotify(syncEx);
final Edit notifyEdit = edit;
final RuntimeException ex = syncEx;
logSyncNotifyExecutor.submit(() -> notifyEdit.logSyncNotify(ex));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5050,6 +5050,18 @@
</description>
</property>

<property>
<name>dfs.namenode.edits.async.logsyncnotify.executor.size</name>
<value>10</value>
<description>
The size of the thread pool executing logSyncNotify in FSEditLogAsync.
This thread pool decouples the RPC response network sending from
the FSEditLogAsync thread so that FSEditLogAsync would not be affected
by the potential network hanging issue (HDFS-15869).
This property defaults to 10.
</description>
</property>

<property>
<name>dfs.namenode.edits.dir.minimum</name>
<value>1</value>
Expand Down