Skip to content

Commit

Permalink
Issue 1544: ConcurrentModificationException with nonblocking logReade…
Browse files Browse the repository at this point in the history
…r.readNext(true)

*Motivation*

Fixes apache#1544. ConcurrentModificationException is thrown when trying to access a non-thread-safe hashmap from different threads.

*Changes*

Make sure accessing to this non-thread-safe hashmap is under synchronized block.

*Tests*

It is a bit tricky to reproduce this race condition in a unit test or an integration test. so not going to attempt adding any tests.
  • Loading branch information
sijie committed Jul 18, 2018
1 parent 7a2e731 commit ccedc76
Showing 1 changed file with 12 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -476,8 +476,12 @@ void notifyLogStreamDeleted(String logSegmentsPath,
return;
}
this.submitTask(logSegmentsPath, () -> {
for (LogSegmentNamesListener listener : listeners.keySet()) {
listener.onLogStreamDeleted();
// the listener map might be updated in different threads (e.g. unregisterLogSegmentListener)
// so access it under a synchronization block
synchronized (listeners) {
for (LogSegmentNamesListener listener : listeners.keySet()) {
listener.onLogStreamDeleted();
}
}
});

Expand All @@ -490,8 +494,12 @@ void notifyLogSegmentsUpdated(String logSegmentsPath,
return;
}
this.submitTask(logSegmentsPath, () -> {
for (VersionedLogSegmentNamesListener listener : listeners.values()) {
listener.onSegmentsUpdated(segments);
// the listener map might be updated in different threads (e.g. unregisterLogSegmentListener)
// so access it under a synchronization block
synchronized (listeners) {
for (VersionedLogSegmentNamesListener listener : listeners.values()) {
listener.onSegmentsUpdated(segments);
}
}
});
}
Expand Down

0 comments on commit ccedc76

Please sign in to comment.