Ver Fonte

HDFS-17711: Change fsimage loading progress percentage discontinuous to continuous

Co-authored-by: Sungdong Kim <cafri.sun@kakaocorp.com>
Signed-off-by: Chris Nauroth <cnauroth@apache.org>
Sung Dong Kim há 3 meses atrás
pai
commit
7ce4ea5780

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

@@ -423,6 +423,7 @@ public final class FSImageFormatPBINode {
       CountDownLatch latch = new CountDownLatch(sections.size());
       AtomicInteger totalLoaded = new AtomicInteger(0);
       final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
+      Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, currentStep);
 
       for (int i=0; i < sections.size(); i++) {
         FileSummary.Section s = sections.get(i);
@@ -433,9 +434,7 @@ public final class FSImageFormatPBINode {
         }
         service.submit(() -> {
           try {
-            totalLoaded.addAndGet(loadINodesInSection(ins, null));
-            prog.setCount(Phase.LOADING_FSIMAGE, currentStep,
-                totalLoaded.get());
+            totalLoaded.addAndGet(loadINodesInSection(ins, counter));
           } catch (Exception e) {
             LOG.error("An exception occurred loading INodes in parallel", e);
             exceptions.add(new IOException(e));