-
Notifications
You must be signed in to change notification settings - Fork 9.2k
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
Changes from 3 commits
20be591
85a8948
aaedc51
bbc76cc
ee2e845
6c856f3
3feb558
b751835
3c57512
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 |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -48,6 +50,9 @@ 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 | ||
| 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. | ||
|
|
@@ -63,6 +68,9 @@ class FSEditLogAsync extends FSEditLog implements Runnable { | |
| DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT); | ||
|
|
||
| editPendingQ = new ArrayBlockingQueue<>(editPendingQSize); | ||
|
|
||
| // the thread pool size should be configurable later, and justified with a rationale | ||
| logSyncNotifyExecutor = Executors.newFixedThreadPool(10); | ||
| } | ||
|
|
||
| private boolean isSyncThreadAlive() { | ||
|
|
@@ -117,6 +125,7 @@ void openForWrite(int layoutVersion) throws IOException { | |
| public void close() { | ||
| super.close(); | ||
| stopSyncThread(); | ||
| logSyncNotifyExecutor.shutdown(); | ||
|
||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -251,7 +260,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)); | ||
| } | ||
| } | ||
| } | ||
|
|
||
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.
I prefer to make this improvement be more configurable to use. Can we make thread pool size be configurable in this PR? if the pool size is configured as 0, that means this improvements is disabled.
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.
Sure. What should be the default value? Many users use the default value, so probably we shouldn't set it as 0 by default.
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.
@functioner , we could make 10 as the default pool size.