瀏覽代碼

HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel (#5300)

Reviewed-by: Stephen O'Donnell <sodonnell@apache.org>
Signed-off-by: Chris Nauroth <cnauroth@apache.org>
(cherry picked from commit 04f3573f6ad07c64c1e2cf8a82fb3e9263366541)
Viraj Jasani 2 年之前
父節點
當前提交
63443be5f4

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

@@ -23,9 +23,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -227,8 +227,7 @@ public final class FSImageFormatPBINode {
       LOG.info("Loading the INodeDirectory section in parallel with {} sub-" +
               "sections", sections.size());
       CountDownLatch latch = new CountDownLatch(sections.size());
-      final CopyOnWriteArrayList<IOException> exceptions =
-          new CopyOnWriteArrayList<>();
+      final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
       for (FileSummary.Section s : sections) {
         service.submit(() -> {
           InputStream ins = null;
@@ -237,8 +236,7 @@ public final class FSImageFormatPBINode {
                 compressionCodec);
             loadINodeDirectorySection(ins);
           } catch (Exception e) {
-            LOG.error("An exception occurred loading INodeDirectories in " +
-                "parallel", e);
+            LOG.error("An exception occurred loading INodeDirectories in parallel", e);
             exceptions.add(new IOException(e));
           } finally {
             latch.countDown();
@@ -424,8 +422,7 @@ public final class FSImageFormatPBINode {
       long expectedInodes = 0;
       CountDownLatch latch = new CountDownLatch(sections.size());
       AtomicInteger totalLoaded = new AtomicInteger(0);
-      final CopyOnWriteArrayList<IOException> exceptions =
-          new CopyOnWriteArrayList<>();
+      final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
 
       for (int i=0; i < sections.size(); i++) {
         FileSummary.Section s = sections.get(i);