Skip to content
Merged
Changes from all commits
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 @@ -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;
Expand Down Expand Up @@ -137,7 +141,7 @@ public SegmentLoadDropHandler(
@LifecycleStart
public void start() throws IOException
{
synchronized (lock) {
synchronized (startStopLock) {
if (started) {
return;
}
Expand All @@ -159,7 +163,7 @@ public void start() throws IOException
@LifecycleStop
public void stop()
{
synchronized (lock) {
synchronized (startStopLock) {
if (!started) {
return;
}
Expand Down Expand Up @@ -296,7 +300,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m
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);
}
}
Expand Down Expand Up @@ -423,7 +427,7 @@ private void removeSegment(
public void run()
{
try {
synchronized (lock) {
synchronized (segmentDeleteLock) {
if (segmentsToDelete.remove(segment)) {
segmentManager.dropSegment(segment);

Expand Down