Skip to content

Commit 7ce4ea5

Browse files
Last-remote11Sungdong Kim
and
Sungdong Kim
authored
HDFS-17711: Change fsimage loading progress percentage discontinuous to continuous
Co-authored-by: Sungdong Kim <[email protected]> Signed-off-by: Chris Nauroth <[email protected]>
1 parent 2343bd7 commit 7ce4ea5

File tree

1 file changed

+2
-3
lines changed

1 file changed

+2
-3
lines changed

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

+2-3
Original file line numberDiff line numberDiff line change
@@ -423,6 +423,7 @@ void loadINodeSectionInParallel(ExecutorService service,
423423
CountDownLatch latch = new CountDownLatch(sections.size());
424424
AtomicInteger totalLoaded = new AtomicInteger(0);
425425
final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
426+
Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, currentStep);
426427

427428
for (int i=0; i < sections.size(); i++) {
428429
FileSummary.Section s = sections.get(i);
@@ -433,9 +434,7 @@ void loadINodeSectionInParallel(ExecutorService service,
433434
}
434435
service.submit(() -> {
435436
try {
436-
totalLoaded.addAndGet(loadINodesInSection(ins, null));
437-
prog.setCount(Phase.LOADING_FSIMAGE, currentStep,
438-
totalLoaded.get());
437+
totalLoaded.addAndGet(loadINodesInSection(ins, counter));
439438
} catch (Exception e) {
440439
LOG.error("An exception occurred loading INodes in parallel", e);
441440
exceptions.add(new IOException(e));

0 commit comments

Comments
 (0)