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

Conversation

functioner
Copy link
Contributor

I propose a fix for HDFS-15869.

Copy link
Contributor

@Hexiaoqiao Hexiaoqiao left a comment

Choose a reason for hiding this comment

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

Thanks @functioner for your works here. leave some nit comments. Please take a look. Thanks.

@Override
public void run() {
final LogSyncNotifyThread logSyncNotifyThread = new LogSyncNotifyThread();
logSyncNotifyThread.start();
Copy link
Contributor

Choose a reason for hiding this comment

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

single thread to push the result back to client here, I think it is safe to trigger multi-thread to do that. right?

EditSyncEx editSyncEx = logSyncNotifyQ.take();
editSyncEx.edit.logSyncNotify(editSyncEx.ex);
}
} catch(InterruptedException ie) {} // just swallow it
Copy link
Contributor

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.

@functioner
Copy link
Contributor Author

Thanks @Hexiaoqiao for your review. I have added the multi-threaded executor for it.

For the exception handling part, I observe the original semantics of RpcEdit

public void logSyncNotify(RuntimeException syncEx) {
try {
if (syncEx == null) {
call.sendResponse();
} else {
call.abortResponse(syncEx);
}
} catch (Exception e) {} // don't care if not sent.
}

and SyncEdit
public void logSyncNotify(RuntimeException ex) {
synchronized(lock) {
done = true;
syncEx = ex;
lock.notifyAll();
}
}

Therefore, we don't need to add any extra handling mechanism.

I have checked the test of TestEditLogRace and TestEditLog in my local machine. It passes all the tests. Let's see if there are other tests that fail.

@Hexiaoqiao
Copy link
Contributor

Hexiaoqiao commented Mar 25, 2021

@functioner would you mind to add unit test to cover this improvement?
cc @jojochuang @ayushtkn @linyiqun I believe this is a great improvement especially for heavy response. Would you mind to take another review. Thanks.

@functioner
Copy link
Contributor Author

@functioner would you mind to add unit test to cover this improvement?

I'm writing a unit test to cover this improvement. After I read the test cases of TestEditLog and TestEditLogRace, I think we can try to add a test similar to TestEditLogRace#testDeadlock. My current design is basically adding an Edit which intentionally sleeps for a while in its logSyncNotify method, blocking the next Edit, and test if the next Edit can finish soon.

However, if we use this design, the test needs to access some private methods and classes in FSEditLogAsync. I think maybe it's not a good idea to change some modifiers in FSEditLogAsync. Can we use something like @exposeToTest? (sorry that I don't have experience in this part). Or can you come up with a better design? @Hexiaoqiao @jojochuang @ayushtkn @linyiqun Thanks.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 55s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
-1 ❌ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
+1 💚 mvninstall 32m 33s trunk passed
+1 💚 compile 1m 20s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 compile 1m 13s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 checkstyle 1m 1s trunk passed
+1 💚 mvnsite 1m 22s trunk passed
+1 💚 javadoc 0m 55s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 25s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 8s trunk passed
+1 💚 shadedclient 16m 14s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 10s the patch passed
+1 💚 compile 1m 9s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javac 1m 9s the patch passed
+1 💚 compile 1m 6s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 javac 1m 6s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 52s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)
+1 💚 mvnsite 1m 11s the patch passed
+1 💚 javadoc 0m 45s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 19s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 7s the patch passed
+1 💚 shadedclient 16m 3s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 730m 2s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 45s The patch does not generate ASF License warnings.
815m 32s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestDecommission
hadoop.hdfs.TestDistributedFileSystemWithECFile
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.TestPersistBlocks
hadoop.hdfs.TestDFSShell
hadoop.hdfs.server.namenode.ha.TestStandbyIsHot
hadoop.hdfs.server.balancer.TestBalancerWithHANameNodes
hadoop.hdfs.server.mover.TestMover
hadoop.hdfs.server.namenode.TestFsck
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.TestFetchImage
hadoop.hdfs.TestDistributedFileSystemWithECFileWithRandomECPolicy
hadoop.hdfs.server.namenode.ha.TestPendingCorruptDnMessages
hadoop.hdfs.server.namenode.ha.TestStandbyBlockManagement
hadoop.hdfs.server.namenode.TestListOpenFiles
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.TestDecommissionWithBackoffMonitor
hadoop.hdfs.tools.TestDFSAdminWithHA
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.TestDFSInotifyEventInputStreamKerberized
hadoop.hdfs.server.namenode.TestNamenodeRetryCache
hadoop.hdfs.server.balancer.TestBalancerService
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.hdfs.server.datanode.TestBlockScanner
hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand
hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots
hadoop.hdfs.server.namenode.TestStoragePolicySatisfierWithHA
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.web.TestWebHdfsFileSystemContract
hadoop.fs.viewfs.TestNNStartupWhenViewFSOverloadSchemeEnabled
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/2/artifact/out/Dockerfile
GITHUB PR #2737
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell
uname Linux 36ed91d4be9c 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / aaedc51
Default Java Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/2/testReport/
Max. process+thread count 2867 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/2/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@@ -63,6 +68,9 @@
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);
Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Copy link
Contributor

@Hexiaoqiao Hexiaoqiao left a comment

Choose a reason for hiding this comment

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

@functioner any plan to push this improvement forward?

@@ -117,6 +125,7 @@ void openForWrite(int layoutVersion) throws IOException {
public void close() {
super.close();
stopSyncThread();
logSyncNotifyExecutor.shutdown();
Copy link
Contributor

Choose a reason for hiding this comment

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

It could meet issue when transition active to standby then transition back here. Because executor has been shutdown and no response will be sent. Moreover I suspect that namenode process could fatal in this case. In my opinion, we do not need to shutdown this executor. FYI. Thanks.

@functioner
Copy link
Contributor Author

@Hexiaoqiao Thanks for the comment, I removed logSyncNotifyExecutor.shutdown().

I think we also need to add some comments in the code to show why we don't shutdown this executor, in case some developers may get confused, because at the first glance it's also normal for people to think that this executor should shutdown on close.

Furthermore, I think your argument implies that the FSEditLogAsync is singleton in terms of the namenode process, otherwise we may create multiple executors without shutting down any of them. Is FSEditLogAsync really always singleton (now & future)?

@Hexiaoqiao
Copy link
Contributor

I think we also need to add some comments in the code to show why we don't shutdown this executor, in case some developers may get confused, because at the first glance it's also normal for people to think that this executor should shutdown on close.

+1.

Furthermore, I think your argument implies that the FSEditLogAsync is singleton in terms of the namenode process, otherwise we may create multiple executors without shutting down any of them. Is FSEditLogAsync really always singleton (now & future)?

IMO, it is true that FSEditLogAsync is singleton instance for NameNode process.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 36s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
-1 ❌ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
+1 💚 mvninstall 34m 1s trunk passed
+1 💚 compile 1m 21s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 compile 1m 16s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 checkstyle 1m 2s trunk passed
+1 💚 mvnsite 1m 19s trunk passed
+1 💚 javadoc 0m 54s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 24s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 4s trunk passed
+1 💚 shadedclient 16m 4s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 16m 23s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 12s the patch passed
+1 💚 compile 1m 13s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javac 1m 13s the patch passed
+1 💚 compile 1m 9s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 javac 1m 9s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 52s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)
+1 💚 mvnsite 1m 11s the patch passed
+1 💚 javadoc 0m 45s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 16s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 6s the patch passed
+1 💚 shadedclient 15m 52s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 231m 29s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 45s The patch does not generate ASF License warnings.
317m 56s
Reason Tests
Failed junit tests hadoop.hdfs.server.balancer.TestBalancer
hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks
hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys
hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots
hadoop.hdfs.server.datanode.TestDirectoryScanner
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/3/artifact/out/Dockerfile
GITHUB PR #2737
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell
uname Linux f6092b2b51d2 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / bbc76cc
Default Java Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/3/testReport/
Max. process+thread count 3144 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/3/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@Hexiaoqiao
Copy link
Contributor

@functioner after review code, comment from Yiqun seems not resolved yet, do you mind to make thread pool size be configurable?

@functioner
Copy link
Contributor Author

@Hexiaoqiao thanks for the reminder. I've added:

  1. comments for removing logSyncNotifyExecutor.shutdown()
  2. configuration for the size of logSyncNotifyExecutor
  3. the docs for that configuration

Copy link
Contributor

@Hexiaoqiao Hexiaoqiao left a comment

Choose a reason for hiding this comment

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

Thanks @functioner for your works. LGTM, +1 from my side. Let's wait for what Yetus says.
Ping @linyiqun do you mind to take another reviews?

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 0s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
-1 ❌ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
+1 💚 mvninstall 34m 44s trunk passed
+1 💚 compile 1m 23s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 compile 1m 15s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 checkstyle 1m 1s trunk passed
+1 💚 mvnsite 1m 21s trunk passed
+1 💚 javadoc 0m 55s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 26s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 11s trunk passed
+1 💚 shadedclient 16m 57s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 17m 15s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 13s the patch passed
+1 💚 compile 1m 21s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javac 1m 21s the patch passed
+1 💚 compile 1m 9s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 javac 1m 9s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 58s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 3 new + 463 unchanged - 0 fixed = 466 total (was 463)
+1 💚 mvnsite 1m 14s the patch passed
+1 💚 xml 0m 1s The patch has no ill-formed XML file.
+1 💚 javadoc 0m 47s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 17s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 22s the patch passed
+1 💚 shadedclient 17m 12s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 417m 5s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 41s The patch does not generate ASF License warnings.
506m 48s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS
hadoop.hdfs.TestPersistBlocks
hadoop.hdfs.TestDFSShell
hadoop.hdfs.TestLeaseRecovery
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.TestLeaseRecovery2
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.hdfs.server.datanode.TestBlockScanner
hadoop.hdfs.server.datanode.TestIncrementalBrVariations
hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots
hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys
hadoop.hdfs.server.datanode.TestDirectoryScanner
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.server.namenode.ha.TestBootstrapStandby
hadoop.hdfs.web.TestWebHdfsFileSystemContract
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/4/artifact/out/Dockerfile
GITHUB PR #2737
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux da07e6c90844 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / b751835
Default Java Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/4/testReport/
Max. process+thread count 3148 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/4/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 0s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
-1 ❌ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
+1 💚 mvninstall 33m 57s trunk passed
+1 💚 compile 1m 19s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 compile 1m 14s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 checkstyle 1m 3s trunk passed
+1 💚 mvnsite 1m 20s trunk passed
+1 💚 javadoc 0m 54s trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 23s trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 5s trunk passed
+1 💚 shadedclient 16m 2s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 16m 20s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 9s the patch passed
+1 💚 compile 1m 11s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javac 1m 11s the patch passed
+1 💚 compile 1m 5s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 javac 1m 5s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 54s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 3 new + 463 unchanged - 0 fixed = 466 total (was 463)
+1 💚 mvnsite 1m 18s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 0m 48s the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 1m 16s the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
+1 💚 spotbugs 3m 10s the patch passed
+1 💚 shadedclient 16m 2s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 458m 45s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 45s The patch does not generate ASF License warnings.
545m 26s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.TestPersistBlocks
hadoop.hdfs.TestDFSShell
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme
hadoop.hdfs.server.blockmanagement.TestAvailableSpaceRackFaultTolerantBPP
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.fs.TestSymlinkHdfsFileSystem
hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.datanode.TestBlockScanner
hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand
hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots
hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys
hadoop.hdfs.server.datanode.TestDirectoryScanner
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.server.namenode.ha.TestBootstrapStandby
hadoop.hdfs.web.TestWebHdfsFileSystemContract
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/5/artifact/out/Dockerfile
GITHUB PR #2737
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 5028b1276e33 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 3c57512
Default Java Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/5/testReport/
Max. process+thread count 3044 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2737/5/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@Hexiaoqiao
Copy link
Contributor

@functioner would you mind to submit addendum blank commit to trigger CI again. Just notice that there are many failed unit tests but it seems not related to this PR(not very sure), Let's wait what Yetus says.
@linyiqun @daryn-sharp @amahussein Do you mind give another reviews here? if no more other comments. I would like to commit it shortly.

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.

@linyiqun
Copy link
Contributor

@functioner would you mind to submit addendum blank commit to trigger CI again. Just notice that there are many failed unit tests but it seems not related to this PR(not very sure), Let's wait what Yetus says.
@linyiqun @daryn-sharp @amahussein Do you mind give another reviews here? if no more other comments. I would like to commit it shortly.

@functioner @Hexiaoqiao , I have another comment for logSyncNotifyExecutor close, please see my above comment.

@amahussein
Copy link
Contributor

The discussion on PR #2878 and HDFS-15957 sounds to be around the same concept.

By definition, if there is a "network issue", the NameNode in general because all RPCs will be problematic. So what makes that `call.sendResponse(); call get a special treatment compared RPC in the NameNode?
I do not see a bug.

  • Network issues are not fixed by delegating the notifications to a pool of thread. This hides the symptoms of the problems and makes it way difficult to the detect the original problem.
  • This code is potentially adding a major overhead because of the lambda expression that implies triggering a sequence of object allocations inside each iteration. "More allocations" lead to: slow executions, means more GC, more frequent Full-GC, ..etc.

-1 on this patch.

@functioner
Copy link
Contributor Author

@amahussein Thanks for the comment.
Can I send an email to you to explain more about the issue? @Hexiaoqiao and I have some more discussion on it, and some discussion is inconvenient to put in here. You can contact me via oierwhz@gmail.com or haoze@jhu.edu and then I will reply.

@Hexiaoqiao
Copy link
Contributor

@amahussein Thanks for your quick response. I think this is not the same concept/issue between HDFS-15957 and HDFS-15869. I have leave comment at HDFS-15869 and suggest to change it to improvement rather than bug fix; (cc @functioner)
IMO, this is a classic Producer-Consumer problem, and it is natural idea to improve performance using parallel way. And Yiqun has reported the same issue at HDFS-15486. In my own production env it was about 5% E2E saving time for write operation.
My suggestion,
A. Update description for improvement rather than bug fix.
B. If any concerns about lambda expression, we could improve it rather than reject it directly.
Welcome any more discussion. Thanks everyone here.

@amahussein
Copy link
Contributor

@amahussein Thanks for your quick response. I think this is not the same concept/issue between HDFS-15957 and HDFS-15869. I have leave comment at HDFS-15869 and suggest to change it to improvement rather than bug fix; (cc @functioner)
IMO, this is a classic Producer-Consumer problem, and it is natural idea to improve performance using parallel way. And Yiqun has reported the same issue at HDFS-15486. In my own production env it was about 5% E2E saving time for write operation.
My suggestion,
A. Update description for improvement rather than bug fix.
B. If any concerns about lambda expression, we could improve it rather than reject it directly.
Welcome any more discussion. Thanks everyone here.

Thanks @Hexiaoqiao Hexiaoqiao for the comment.
Ok, is the purpose of the change is to improve performance of the FSEditLogAsync.java by executing sendResponse() in parallel?
In that case, please change the title of the Jira and the description to remove references to "hanging" problems.
Then I will take another look. I am sorry for the inconvenience as I want to make sure I understand the purpose of the change before revieweing.

@amahussein Thanks for the comment.
Can I send an email to you to explain more about the issue? @Hexiaoqiao and I have some more discussion on it, and some discussion is inconvenient to put in here. You can contact me via oierwhz@gmail.com or haoze@jhu.edu and then I will reply.

Thanks @functioner ! I really appreciate that.
I think @Hexiaoqiao Hexiaoqiao reply already clarified some of the confusion about the scope of the work.
Please feel free to reach me through email at anytime. I am on the common-dev mailing list.

@functioner
Copy link
Contributor Author

If any concerns about lambda expression, we could improve it rather than reject it directly.

@amahussein A common way to eliminate such overhead is preparing multiple consumer threads, and feed them with requests.
If the lambda expressions cause significant overhead, we can improve in that way.
This design pattern is widely used in Cassandra. Example: SEPWorker - SEPExecutor
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPWorker.java
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPExecutor.java

@functioner
Copy link
Contributor Author

In that case, please change the title of the Jira and the description to remove references to "hanging" problems.

@amahussein I still would like to argue about this "hanging" issue.

There has been reported TCP network I/O issues which hangs for >15min without throwing any exception. ZOOKEEPER-2201 is a perfect example, and you can find the TCP level explanation for this hanging issue in https://www.usenix.org/conference/srecon16/program/presentation/nadolny
Similar hanging bugs are also accepted by ZooKeeper community, such as:

However, in our scenario (HDFS-15869), a possible counterargument is: the call.sendResponse() invocation eventually invokes channel.write(buffer) (line 3611), which is non-blocking mode, so it might not be affected by this potential issue.

private int channelWrite(WritableByteChannel channel,
ByteBuffer buffer) throws IOException {
int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
channel.write(buffer) : channelIO(null, channel, buffer);
if (count > 0) {
rpcMetrics.incrSentBytes(count);
}
return count;
}

However, as we point out in HDFS-15869, when the payload is huge, in line 3611, it won't invoke channel.write(buffer); instead, it invokes channelIO(null, channel, buffer) which brings us to:
private static int channelIO(ReadableByteChannel readCh,
WritableByteChannel writeCh,
ByteBuffer buf) throws IOException {
int originalLimit = buf.limit();
int initialRemaining = buf.remaining();
int ret = 0;
while (buf.remaining() > 0) {
try {
int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
buf.limit(buf.position() + ioSize);
ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
if (ret < ioSize) {
break;
}
} finally {
buf.limit(originalLimit);
}
}
int nBytes = initialRemaining - buf.remaining();
return (nBytes > 0) ? nBytes : ret;
}

If the payload is split in two batches, the second batch will have to wait for the first batch to be sent out, which may encounter high packet loss rate and thus slow I/O.

Hence, I would say the hanging problem still exists.

@functioner
Copy link
Contributor Author

In that case, please change the title of the Jira and the description to remove references to "hanging" problems.

@amahussein I still would like to argue about this "hanging" issue.

Another aspect of the argument is the design of availability and fault tolerance. Actually distributed systems can tolerate such hanging issues in many scenarios, but sometimes it's seen as a bug like ZOOKEEPER-2201.

So an important question is: when it's a bug; and when it's not (i.e., it's a feature)

I've been doing research on fault injection for some time and I have submitted multiple bug reports accepted by the open source community (e.g., HADOOP-17552). My criteria for determining whether it is bug, are:

  1. if we inject a fault in module X and it only affects module X, then it’s not a bug.
  2. if we inject a fault in module X and it affects not only module X but also module Y which should not relate to module X, then probably it would be a bug, because in the system design, each module should be responsible for itself and report the problem (e.g., by logging), rather than affect another irrelevant module.

In our scenario (HDFS-15869), this possible hanging (if you agree with my argument of network hanging) can block the FSEditLogAsync thread, because now call.sendResponse() is invoked by the FSEditLogAsync thread.

So, call.sendResponse() (network service) affects FSEditLogAsync (edit log sync service). So, I would say it's a bug.

The network service should be responsible for all its behaviors, and handle all the possible network issues (e.g., IOException, disconnection, hanging). It should determine how to handle them, e.g., by logging the error, rather than affecting other services like FSEditLogAsync.

I'm not saying that we have to use a complete and slow RPC framework for this network service. But IMO, decoupling it from FSEditLogAsync by delegating to a thread pool is at least a better design.

@amahussein
Copy link
Contributor

amahussein commented Apr 19, 2021

Thanks @functioner
The detailed discussions (except the lambda argument) should have been on the Jira.

IMO, this is a classic Producer-Consumer problem, and it is natural idea to improve performance using parallel way.

So, call.sendResponse() (network service) affects FSEditLogAsync (edit log sync service). So, I would say it's a bug.

Now, I am really even more confused about the (Bug Vs. Improvement). So, I am going to pass on reviewing.

If any concerns about lambda expression, we could improve it rather than reject it directly.

@amahussein A common way to eliminate such overhead is preparing multiple consumer threads, and feed them with requests.
If the lambda expressions cause significant overhead, we can improve in that way.
This design pattern is widely used in Cassandra. Example: SEPWorker - SEPExecutor
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPWorker.java
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPExecutor.java

This is not what I meant. It is recommended to avoid use of lambda expressions in hot execution paths.
There are so many ways to avoid lambda expressions simply by having runnables waiting for tasks to be added to a queue.

@functioner
Copy link
Contributor Author

Thanks @functioner
The detailed discussions (except the lambda argument) should have been on the Jira.

I see. I will make a comment of summary in Jira after the discussion in this PR is finalized.

If any concerns about lambda expression, we could improve it rather than reject it directly.

@amahussein A common way to eliminate such overhead is preparing multiple consumer threads, and feed them with requests.
If the lambda expressions cause significant overhead, we can improve in that way.
This design pattern is widely used in Cassandra. Example: SEPWorker - SEPExecutor
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPWorker.java
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPExecutor.java

This is not what I meant. It is recommended to avoid use of lambda expressions in hot execution paths.

Actually we are on the same page. Maybe my comment has made some confusion.

There are so many ways to avoid lambda expressions simply by having runnables waiting for tasks to be added to a queue.

That's exactly what I meant. I will push a commit soon.

@functioner
Copy link
Contributor Author

functioner commented Apr 19, 2021

Thanks @functioner
The detailed discussions (except the lambda argument) should have been on the Jira.

IMO, this is a classic Producer-Consumer problem, and it is natural idea to improve performance using parallel way.

So, call.sendResponse() (network service) affects FSEditLogAsync (edit log sync service). So, I would say it's a bug.

Now, I am really even more confused about the (Bug Vs. Improvement). So, I am going to pass on reviewing.

@amahussein Thanks for your feedback, and your time!
Sorry for all the possible confusion I made.

It's not a big deal whether it's marked as bug or improvement. One of my bug reports (HADOOP-17552) is also finally marked as improvement rather than bug. The point is that the developers (in HADOOP-17552) finally realized that there's a potential hanging issue as I point out, and the patch (as well as the relevant discussion) is very helpful for the developers and the users.

Ok, is the purpose of the change is to improve performance of the FSEditLogAsync.java by executing sendResponse() in parallel?
In that case, please change the title of the Jira and the description to remove references to "hanging" problems.

My intention is to defend that we should not remove the references to "hanging" problems.

In short, the discussion above can be summarized into 3 arguments:

  1. HDFS-15869. Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang #2737 (comment): this is a classic Producer-Consumer problem, and it is natural idea to improve performance using parallel way
  2. HDFS-15869. Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang #2737 (comment): the call.sendResponse() may hang due to network issue, without throwing any exception
  3. HDFS-15869. Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang #2737 (comment):
    a). the FSEditLogAsync thread (without this patch) directly invokes a network I/O call call.sendResponse(), so if this network I/O invocation hangs, the FSEditLogAsync thread also hangs
    b). in the "correct" system design, if this network I/O invocation hangs in this way, then that should be fine, because HDFS (as a fault-tolerant system) should tolerate it.
    c). when the system tolerates this network issue, the FSEditLogAsync thread should not hang, otherwise everybody can't commit the log.
    d). our expected behavior is that, when the system tolerates this network issue, the FSEditLogAsync thread should continue, so that everything still works well, despite this network issue.

Both Argument 3 and Argument 1 can be resolved with this patch.

In conclusion, this patch not only improves the performance, but also enhances the availability & fault-tolerance.

So, I think the references to "hanging" problems should not be removed.

If it keeps "Improvement" tag instead of "Bug" tag, that's fine.

P.S. I will summarize our discussion with a comment in Jira after we reach a consensus.

@daryn-sharp
Copy link
Contributor

daryn-sharp commented Apr 19, 2021

the FSEditLogAsync thread (without this patch) directly invokes a network I/O call call.sendResponse(), so if this network I/O invocation hangs, the FSEditLogAsync thread also hangs
My intention is to defend that we should not remove the references to "hanging" problems.

No, it's a non-blocking write so by definition it will never hang – unless induced by fault injection.

@functioner
Copy link
Contributor Author

functioner commented Apr 19, 2021

No, it's a non-blocking write so by definition it will never hang – unless induced by fault injection.

@daryn-sharp I have considered this counterargument in #2737 (comment), where I proposed another argument that it may hang when it's a huge payload, because there's a while-loop. Please take a look.

Yes, channel.write(buffer) is non-blocking. But I suspect that channelIO(null, channel, buffer) is blocking, otherwise we won't have the performance issue in HDFS-15486. Within channelIO(null, channel, buffer), the large payload is split into multiple parts, and it will won't jump out of the loop until the remaining part of payload does not exceed the buffer limit, meaning that it's waiting for the network to finish sending some content.

I'm not sure whether it can defend my argument. Can you provide more explanation? Maybe I'm not correct. Thanks!

@ZanderXu
Copy link
Contributor

@functioner Can you move forward with this MR? I can help with review. If you don't have time, I can resign this ticket to other contributor.

@functioner
Copy link
Contributor Author

@functioner Can you move forward with this MR? I can help with review. If you don't have time, I can resign this ticket to other contributor.

@ZanderXu According to the discussion so far, Daryn has some doubt on the issue and patch. Do you have any idea to move forward the case?

@ZanderXu
Copy link
Contributor

Thanks for your works and discussions for this problem. I spent a long time to catch your ideas and concerns😂, it's so hard.

I have some throughs and questions about this ticket.

Some questions in HDFS-15869:

The channel.write(buffer) operation in line 3594 may be slow. Although for this specific stack trace, the channel is initialized in the non-blocking mode, there is still a chance of being slow depending on native write implementation in the OS (e.g., a kernel issue). Furthermore, the channelIO invocation in line 3594 may also get stuck, since it waits until the buffer is drained:

ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf); will return 0 if namenode cannot write more data to this connection, right? RpcEdit,logSyncNotify will add this response into the queue of this connection and let the Responder to take this job, right? So FSEditLogAsync can go to process the next jobs, right?

Some throughs in HDFS-15869:

Actually, I encountered this problems in our prod environment that the thread FSEditLogAsync spends a little more time to send responses to clients, which had a big performance impact on writing edits to JNs. So I just use a new single thread to do these send response jobs. Of course, we can use multiple threads to send responses to client. New task is very expensive, so we use a producer-consumer mode to fix this problem.

  • FSEditLogAsync just put task into a capacity blocking Queue.
  • ResponseSender thread take tasks from the Queue and send them to clients.

About "Bug" or "Improvement", I think it should be a performance improvement, since all processes are worked as expected, no blocking or hanging, just slow.

Some throughs in HDFS-15957:

  • I think namenode should directly close this connection if IOException happens in RpcEdit,logSyncNotify, since we cannot let the client hang forever. It seems that the namenode drops a request.

@functioner Looking forward your ideas and confirm.

@daryn-sharp @Hexiaoqiao @linyiqun @amahussein Looking forward your ideas. I hope we can push this ticket forward.

@ZanderXu
Copy link
Contributor

If I missed some other concerns, please let me know, we can find solutions together to push this ticket forward.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants