This is an automated email from the ASF dual-hosted git repository. cnauroth pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/branch-3.3 by this push: new 63443be5f4d HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel (#5300) 63443be5f4d is described below commit 63443be5f4ddb2751c0bb1898362b4be8f9909a2 Author: Viraj Jasani <vjas...@apache.org> AuthorDate: Wed Jan 18 13:13:41 2023 -0800 HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel (#5300) Reviewed-by: Stephen O'Donnell <sodonn...@apache.org> Signed-off-by: Chris Nauroth <cnaur...@apache.org> (cherry picked from commit 04f3573f6ad07c64c1e2cf8a82fb3e9263366541) --- .../hadoop/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 0a69c99cab8..26df995e552 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.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); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org