SegmentLoadDropHandler: Fix deadlock when segments have errors loading on startup. (#5735)

The "lock" object was used to synchronize start/stop as well as synchronize removals
from segmentsToDelete (when a segment is done dropping). This could cause a deadlock
if a segment-load throws an exception during loadLocalCache. loadLocalCache is run
by start() while it holds the lock, but then it spawns loading threads, and those
threads will try to acquire the "segmentsToDelete" lock if they want to drop a corrupt
segments.

I don't see any reason for these two locks to be the same lock, so I split them.
This commit is contained in:
Gian Merlino 2018-05-03 09:59:01 -07:00 committed by Slim Bouguerra
parent d2b8d880ea
commit df01998213
1 changed files with 9 additions and 5 deletions

View File

@ -73,7 +73,11 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
{
private static final EmittingLogger log = new EmittingLogger(SegmentLoadDropHandler.class);
private final Object lock = new Object();
// Synchronizes removals from segmentsToDelete
private final Object segmentDeleteLock = new Object();
// Synchronizes start/stop of this object.
private final Object startStopLock = new Object();
private final ObjectMapper jsonMapper;
private final SegmentLoaderConfig config;
@ -137,7 +141,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
@LifecycleStart
public void start() throws IOException
{
synchronized (lock) {
synchronized (startStopLock) {
if (started) {
return;
}
@ -159,7 +163,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
@LifecycleStop
public void stop()
{
synchronized (lock) {
synchronized (startStopLock) {
if (!started) {
return;
}
@ -296,7 +300,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
things slow. Given that in most cases segmentsToDelete.contains(segment) returns false, it will save a lot of
cost of acquiring lock by doing the "contains" check outside the synchronized block.
*/
synchronized (lock) {
synchronized (segmentDeleteLock) {
segmentsToDelete.remove(segment);
}
}
@ -423,7 +427,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
public void run()
{
try {
synchronized (lock) {
synchronized (segmentDeleteLock) {
if (segmentsToDelete.remove(segment)) {
segmentManager.dropSegment(segment);