Skip to content

Commit 04f3573

Browse files
authored
HDFS-16891 Avoid the overhead of copy-on-write exception list while loading inodes sub sections in parallel (#5300)
Reviewed-by: Stephen O'Donnell <[email protected]> Signed-off-by: Chris Nauroth <[email protected]>
1 parent 442a5fb commit 04f3573

File tree

1 file changed

+4
-7
lines changed

1 file changed

+4
-7
lines changed

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

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,9 @@
2323
import java.io.OutputStream;
2424
import java.util.ArrayList;
2525
import java.util.Collection;
26+
import java.util.Collections;
2627
import java.util.Iterator;
2728
import java.util.List;
28-
import java.util.concurrent.CopyOnWriteArrayList;
2929
import java.util.concurrent.CountDownLatch;
3030
import java.util.concurrent.ExecutorService;
3131
import java.util.concurrent.Executors;
@@ -227,8 +227,7 @@ void loadINodeDirectorySectionInParallel(ExecutorService service,
227227
LOG.info("Loading the INodeDirectory section in parallel with {} sub-" +
228228
"sections", sections.size());
229229
CountDownLatch latch = new CountDownLatch(sections.size());
230-
final CopyOnWriteArrayList<IOException> exceptions =
231-
new CopyOnWriteArrayList<>();
230+
final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
232231
for (FileSummary.Section s : sections) {
233232
service.submit(() -> {
234233
InputStream ins = null;
@@ -237,8 +236,7 @@ void loadINodeDirectorySectionInParallel(ExecutorService service,
237236
compressionCodec);
238237
loadINodeDirectorySection(ins);
239238
} catch (Exception e) {
240-
LOG.error("An exception occurred loading INodeDirectories in " +
241-
"parallel", e);
239+
LOG.error("An exception occurred loading INodeDirectories in parallel", e);
242240
exceptions.add(new IOException(e));
243241
} finally {
244242
latch.countDown();
@@ -424,8 +422,7 @@ void loadINodeSectionInParallel(ExecutorService service,
424422
long expectedInodes = 0;
425423
CountDownLatch latch = new CountDownLatch(sections.size());
426424
AtomicInteger totalLoaded = new AtomicInteger(0);
427-
final CopyOnWriteArrayList<IOException> exceptions =
428-
new CopyOnWriteArrayList<>();
425+
final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
429426

430427
for (int i=0; i < sections.size(); i++) {
431428
FileSummary.Section s = sections.get(i);

0 commit comments

Comments
 (0)