-
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
Closed
Closed
Changes from 1 commit
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
20be591
HDFS-15869. Network issue while FSEditLogAsync is executing RpcEdit.l…
functioner 85a8948
Merge branch 'trunk' into HDFS-15869
functioner aaedc51
add multi-threaded executor
functioner bbc76cc
remove executor shutdown
functioner ee2e845
Merge branch 'trunk' into HDFS-15869
functioner 6c856f3
add comments for logSyncNotifyExecutor
functioner 3feb558
make logSyncNotifyExecutor size configurable
functioner b751835
add docs for the configuration
functioner 3c57512
remove a useless blank line
functioner File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import java.util.List; | ||
| import java.util.concurrent.ArrayBlockingQueue; | ||
| import java.util.concurrent.BlockingQueue; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.Semaphore; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
|
|
@@ -47,6 +48,7 @@ class FSEditLogAsync extends FSEditLog implements Runnable { | |
|
|
||
| // requires concurrent access from caller threads and syncing thread. | ||
| private final BlockingQueue<Edit> editPendingQ; | ||
| private final BlockingQueue<EditSyncEx> logSyncNotifyQ; | ||
|
|
||
| // only accessed by syncing thread so no synchronization required. | ||
| // queue is unbounded because it's effectively limited by the size | ||
|
|
@@ -63,6 +65,7 @@ class FSEditLogAsync extends FSEditLog implements Runnable { | |
| DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT); | ||
|
|
||
| editPendingQ = new ArrayBlockingQueue<>(editPendingQSize); | ||
| logSyncNotifyQ = new LinkedBlockingQueue<>(); | ||
| } | ||
|
|
||
| private boolean isSyncThreadAlive() { | ||
|
|
@@ -227,8 +230,33 @@ private Edit dequeueEdit() throws InterruptedException { | |
| return syncWaitQ.isEmpty() ? editPendingQ.take() : editPendingQ.poll(); | ||
| } | ||
|
|
||
| private static class EditSyncEx { | ||
| final Edit edit; | ||
| final RuntimeException ex; | ||
| EditSyncEx(Edit edit, RuntimeException ex) { | ||
| this.edit = edit; | ||
| this.ex = ex; | ||
| } | ||
| } | ||
|
|
||
| private class LogSyncNotifyThread extends Thread { | ||
| volatile boolean stopped = false; | ||
|
|
||
| @Override | ||
| public void run() { | ||
| try { | ||
| while (!stopped) { | ||
| EditSyncEx editSyncEx = logSyncNotifyQ.take(); | ||
| editSyncEx.edit.logSyncNotify(editSyncEx.ex); | ||
| } | ||
| } catch(InterruptedException ie) {} // just swallow it | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void run() { | ||
| final LogSyncNotifyThread logSyncNotifyThread = new LogSyncNotifyThread(); | ||
| logSyncNotifyThread.start(); | ||
|
||
| try { | ||
| while (true) { | ||
| boolean doSync; | ||
|
|
@@ -251,12 +279,14 @@ public void run() { | |
| syncEx = ex; | ||
| } | ||
| while ((edit = syncWaitQ.poll()) != null) { | ||
| edit.logSyncNotify(syncEx); | ||
| logSyncNotifyQ.put(new EditSyncEx(edit, syncEx)); | ||
| } | ||
| } | ||
| } | ||
| } catch (InterruptedException ie) { | ||
| LOG.info(Thread.currentThread().getName() + " was interrupted, exiting"); | ||
| logSyncNotifyThread.stopped = true; | ||
| logSyncNotifyThread.interrupt(); | ||
| } catch (Throwable t) { | ||
| terminate(t); | ||
| } | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
catch InterruptedException and no handle here? does it cause client never get result? IMO, we should set stopped = true here.