From 8d6faadf6d80b26128e1308b7ad987ceb4f61d74 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Fri, 13 Jan 2023 16:20:28 -0800 Subject: [PATCH 1/2] HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel --- .../server/namenode/FSImageFormatPBINode.java | 19 ++++++++++++------- 1 file changed, 12 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 3f0c9faa97c9a..9a01836228b3c 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 @@ -25,7 +25,6 @@ import java.util.Collection; 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 +226,8 @@ 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 = new ArrayList<>(); + final Object exceptionListLock = new Object(); for (FileSummary.Section s : sections) { service.submit(() -> { InputStream ins = null; @@ -239,7 +238,10 @@ void loadINodeDirectorySectionInParallel(ExecutorService service, } catch (Exception e) { LOG.error("An exception occurred loading INodeDirectories in " + "parallel", e); - exceptions.add(new IOException(e)); + IOException exception = new IOException(e); + synchronized (exceptionListLock) { + exceptions.add(exception); + } } finally { latch.countDown(); try { @@ -424,8 +426,8 @@ 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 = new ArrayList<>(); + final Object exceptionListLock = new Object(); for (int i=0; i < sections.size(); i++) { FileSummary.Section s = sections.get(i); @@ -441,7 +443,10 @@ void loadINodeSectionInParallel(ExecutorService service, totalLoaded.get()); } catch (Exception e) { LOG.error("An exception occurred loading INodes in parallel", e); - exceptions.add(new IOException(e)); + IOException exception = new IOException(e); + synchronized (exceptionListLock) { + exceptions.add(exception); + } } finally { latch.countDown(); try { From f205782b8ee24449b5dcf91db00f726398eb7ca0 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 17 Jan 2023 16:20:10 -0800 Subject: [PATCH 2/2] addendum --- .../server/namenode/FSImageFormatPBINode.java | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 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 9a01836228b3c..1f21871ac7b02 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,6 +23,7 @@ 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.CountDownLatch; @@ -226,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 List exceptions = new ArrayList<>(); - final Object exceptionListLock = new Object(); + final List exceptions = Collections.synchronizedList(new ArrayList<>()); for (FileSummary.Section s : sections) { service.submit(() -> { InputStream ins = null; @@ -236,12 +236,8 @@ void loadINodeDirectorySectionInParallel(ExecutorService service, compressionCodec); loadINodeDirectorySection(ins); } catch (Exception e) { - LOG.error("An exception occurred loading INodeDirectories in " + - "parallel", e); - IOException exception = new IOException(e); - synchronized (exceptionListLock) { - exceptions.add(exception); - } + LOG.error("An exception occurred loading INodeDirectories in parallel", e); + exceptions.add(new IOException(e)); } finally { latch.countDown(); try { @@ -426,8 +422,7 @@ void loadINodeSectionInParallel(ExecutorService service, long expectedInodes = 0; CountDownLatch latch = new CountDownLatch(sections.size()); AtomicInteger totalLoaded = new AtomicInteger(0); - final List exceptions = new ArrayList<>(); - final Object exceptionListLock = new Object(); + final List exceptions = Collections.synchronizedList(new ArrayList<>()); for (int i=0; i < sections.size(); i++) { FileSummary.Section s = sections.get(i); @@ -443,10 +438,7 @@ void loadINodeSectionInParallel(ExecutorService service, totalLoaded.get()); } catch (Exception e) { LOG.error("An exception occurred loading INodes in parallel", e); - IOException exception = new IOException(e); - synchronized (exceptionListLock) { - exceptions.add(exception); - } + exceptions.add(new IOException(e)); } finally { latch.countDown(); try {