Skip to content

Commit a55754c

Browse files
author
eddy.cao
committed
Fix serial fsimage transfer during checkpoint with multiple namenodes
1 parent 744088a commit a55754c

File tree

1 file changed

+4
-3
lines changed

1 file changed

+4
-3
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -248,9 +248,10 @@ private void doCheckpoint() throws InterruptedException, IOException {
248248
// Do this in a separate thread to avoid blocking transition to active, but don't allow more
249249
// than the expected number of tasks to run or queue up
250250
// See HDFS-4816
251-
ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100,
252-
TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),
253-
uploadThreadFactory);
251+
ExecutorService executor =
252+
new ThreadPoolExecutor(activeNNAddresses.size(), activeNNAddresses.size(), 100,
253+
TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),
254+
uploadThreadFactory);
254255
// for right now, just match the upload to the nn address by convention. There is no need to
255256
// directly tie them together by adding a pair class.
256257
HashMap<String, Future<TransferFsImage.TransferResult>> uploads =

0 commit comments

Comments
 (0)