You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
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
In our cluster, each namespace has four NameNodes: one active, one standby, and two observers. When the standby NameNode performs a checkpoint, it transfer the fsimage to the other three NameNodes. However, we found that these transfer are performed serially.
The reason is that the corePoolSize in ThreadPoolExecutor is 0, and the transfer task does not fill the LinkedBlockingQueue, resulting in only one thread transfer the fsimage at a time. This greatly increases the checkpoint time. ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()), uploadThreadFactory);
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
26m 46s
trunk passed
+1 💚
compile
0m 44s
trunk passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚
compile
0m 40s
trunk passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚
checkstyle
0m 38s
trunk passed
+1 💚
mvnsite
0m 43s
trunk passed
+1 💚
javadoc
0m 45s
trunk passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚
javadoc
1m 7s
trunk passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚
spotbugs
1m 42s
trunk passed
+1 💚
shadedclient
22m 45s
branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚
mvninstall
0m 33s
the patch passed
+1 💚
compile
0m 37s
the patch passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚
javac
0m 37s
the patch passed
+1 💚
compile
0m 35s
the patch passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚
javac
0m 35s
the patch passed
+1 💚
blanks
0m 0s
The patch has no blanks issues.
+1 💚
checkstyle
0m 28s
the patch passed
+1 💚
mvnsite
0m 39s
the patch passed
+1 💚
javadoc
0m 34s
the patch passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚
javadoc
0m 57s
the patch passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚
spotbugs
1m 37s
the patch passed
+1 💚
shadedclient
22m 8s
patch has no errors when building and testing our client artifacts.
@Hexiaoqiao@ayushtkn@tomscut Do you think that uploading fsimage in checkpoint with observer namenode should be changed from serial to parallel?
In our production cluster, there are one active, one standby, and two oberver namenodes. The serial uploading takes 40 minutes longer than the parallel operation.
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.
In our cluster, each namespace has four NameNodes: one active, one standby, and two observers. When the standby NameNode performs a checkpoint, it transfer the fsimage to the other three NameNodes. However, we found that these transfer are performed serially.
The reason is that the corePoolSize in ThreadPoolExecutor is 0, and the transfer task does not fill the LinkedBlockingQueue, resulting in only one thread transfer the fsimage at a time. This greatly increases the checkpoint time.
ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()), uploadThreadFactory);