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>
This commit is contained in:
parent
442a5fb285
commit
04f3573f6a
@ -23,9 +23,9 @@
|
|||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
@ -227,8 +227,7 @@ void loadINodeDirectorySectionInParallel(ExecutorService service,
|
|||||||
LOG.info("Loading the INodeDirectory section in parallel with {} sub-" +
|
LOG.info("Loading the INodeDirectory section in parallel with {} sub-" +
|
||||||
"sections", sections.size());
|
"sections", sections.size());
|
||||||
CountDownLatch latch = new CountDownLatch(sections.size());
|
CountDownLatch latch = new CountDownLatch(sections.size());
|
||||||
final CopyOnWriteArrayList<IOException> exceptions =
|
final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
|
||||||
new CopyOnWriteArrayList<>();
|
|
||||||
for (FileSummary.Section s : sections) {
|
for (FileSummary.Section s : sections) {
|
||||||
service.submit(() -> {
|
service.submit(() -> {
|
||||||
InputStream ins = null;
|
InputStream ins = null;
|
||||||
@ -237,8 +236,7 @@ void loadINodeDirectorySectionInParallel(ExecutorService service,
|
|||||||
compressionCodec);
|
compressionCodec);
|
||||||
loadINodeDirectorySection(ins);
|
loadINodeDirectorySection(ins);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.error("An exception occurred loading INodeDirectories in " +
|
LOG.error("An exception occurred loading INodeDirectories in parallel", e);
|
||||||
"parallel", e);
|
|
||||||
exceptions.add(new IOException(e));
|
exceptions.add(new IOException(e));
|
||||||
} finally {
|
} finally {
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
@ -424,8 +422,7 @@ void loadINodeSectionInParallel(ExecutorService service,
|
|||||||
long expectedInodes = 0;
|
long expectedInodes = 0;
|
||||||
CountDownLatch latch = new CountDownLatch(sections.size());
|
CountDownLatch latch = new CountDownLatch(sections.size());
|
||||||
AtomicInteger totalLoaded = new AtomicInteger(0);
|
AtomicInteger totalLoaded = new AtomicInteger(0);
|
||||||
final CopyOnWriteArrayList<IOException> exceptions =
|
final List<IOException> exceptions = Collections.synchronizedList(new ArrayList<>());
|
||||||
new CopyOnWriteArrayList<>();
|
|
||||||
|
|
||||||
for (int i=0; i < sections.size(); i++) {
|
for (int i=0; i < sections.size(); i++) {
|
||||||
FileSummary.Section s = sections.get(i);
|
FileSummary.Section s = sections.get(i);
|
||||||
|
Loading…
Reference in New Issue
Block a user