From 04f3573f6ad07c64c1e2cf8a82fb3e9263366541 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 18 Jan 2023 13:13:41 -0800 Subject: [PATCH] HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel (#5300) Reviewed-by: Stephen O'Donnell Signed-off-by: Chris Nauroth --- .../hdfs/server/namenode/FSImageFormatPBINode.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java index 3f0c9faa97..1f21871ac7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java @@ -23,9 +23,9 @@ 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 @@ void loadINodeDirectorySectionInParallel(ExecutorService service, LOG.info("Loading the INodeDirectory section in parallel with {} sub-" + "sections", sections.size()); CountDownLatch latch = new CountDownLatch(sections.size()); - final CopyOnWriteArrayList exceptions = - new CopyOnWriteArrayList<>(); + final List exceptions = Collections.synchronizedList(new ArrayList<>()); for (FileSummary.Section s : sections) { service.submit(() -> { InputStream ins = null; @@ -237,8 +236,7 @@ void loadINodeDirectorySectionInParallel(ExecutorService service, 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 @@ void loadINodeSectionInParallel(ExecutorService service, long expectedInodes = 0; CountDownLatch latch = new CountDownLatch(sections.size()); AtomicInteger totalLoaded = new AtomicInteger(0); - final CopyOnWriteArrayList exceptions = - new CopyOnWriteArrayList<>(); + final List exceptions = Collections.synchronizedList(new ArrayList<>()); for (int i=0; i < sections.size(); i++) { FileSummary.Section s = sections.get(i);