Skip to content
Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<IOException> exceptions =
new CopyOnWriteArrayList<>();
final List<IOException> exceptions = new ArrayList<>();
final Object exceptionListLock = new Object();
for (FileSummary.Section s : sections) {
service.submit(() -> {
InputStream ins = null;
Expand All @@ -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 {
Expand Down Expand Up @@ -424,8 +426,8 @@ void loadINodeSectionInParallel(ExecutorService service,
long expectedInodes = 0;
CountDownLatch latch = new CountDownLatch(sections.size());
AtomicInteger totalLoaded = new AtomicInteger(0);
final CopyOnWriteArrayList<IOException> exceptions =
new CopyOnWriteArrayList<>();
final List<IOException> exceptions = new ArrayList<>();
final Object exceptionListLock = new Object();

for (int i=0; i < sections.size(); i++) {
FileSummary.Section s = sections.get(i);
Expand All @@ -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 {
Expand Down