mirror of https://github.com/apache/druid.git
fix another bug in LQP with concurrency
This commit is contained in:
parent
35937132a7
commit
68827ff88b
|
@ -44,6 +44,7 @@ import java.util.Collection;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
@ -80,10 +81,10 @@ public class LoadQueuePeon
|
|||
private final AtomicLong queuedSize = new AtomicLong(0);
|
||||
private final AtomicInteger failedAssignCount = new AtomicInteger(0);
|
||||
|
||||
private final ConcurrentSkipListSet<SegmentHolder> segmentsToLoad = new ConcurrentSkipListSet<SegmentHolder>(
|
||||
private final ConcurrentSkipListMap<SegmentHolder, SegmentHolder> segmentsToLoad = new ConcurrentSkipListMap<>(
|
||||
segmentHolderComparator
|
||||
);
|
||||
private final ConcurrentSkipListSet<SegmentHolder> segmentsToDrop = new ConcurrentSkipListSet<SegmentHolder>(
|
||||
private final ConcurrentSkipListMap<SegmentHolder, SegmentHolder> segmentsToDrop = new ConcurrentSkipListMap<>(
|
||||
segmentHolderComparator
|
||||
);
|
||||
|
||||
|
@ -111,9 +112,9 @@ public class LoadQueuePeon
|
|||
@JsonProperty
|
||||
public Set<DataSegment> getSegmentsToLoad()
|
||||
{
|
||||
return new ConcurrentSkipListSet<DataSegment>(
|
||||
return new ConcurrentSkipListSet<>(
|
||||
Collections2.transform(
|
||||
segmentsToLoad,
|
||||
segmentsToLoad.keySet(),
|
||||
new Function<SegmentHolder, DataSegment>()
|
||||
{
|
||||
@Override
|
||||
|
@ -129,9 +130,9 @@ public class LoadQueuePeon
|
|||
@JsonProperty
|
||||
public Set<DataSegment> getSegmentsToDrop()
|
||||
{
|
||||
return new ConcurrentSkipListSet<DataSegment>(
|
||||
return new ConcurrentSkipListSet<>(
|
||||
Collections2.transform(
|
||||
segmentsToDrop,
|
||||
segmentsToDrop.keySet(),
|
||||
new Function<SegmentHolder, DataSegment>()
|
||||
{
|
||||
@Override
|
||||
|
@ -169,12 +170,13 @@ public class LoadQueuePeon
|
|||
}
|
||||
}
|
||||
|
||||
SegmentHolder holder = new SegmentHolder(segment, LOAD, Arrays.asList(callback));
|
||||
final SegmentHolder holder = new SegmentHolder(segment, LOAD, Arrays.asList(callback));
|
||||
|
||||
synchronized (lock) {
|
||||
if (segmentsToLoad.contains(holder)) {
|
||||
final SegmentHolder existingHolder = segmentsToLoad.get(holder);
|
||||
if (existingHolder != null) {
|
||||
if ((callback != null)) {
|
||||
holder.addCallback(callback);
|
||||
existingHolder.addCallback(callback);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
@ -182,7 +184,7 @@ public class LoadQueuePeon
|
|||
|
||||
log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier());
|
||||
queuedSize.addAndGet(segment.getSize());
|
||||
segmentsToLoad.add(holder);
|
||||
segmentsToLoad.put(holder, holder);
|
||||
doNext();
|
||||
}
|
||||
|
||||
|
@ -204,16 +206,17 @@ public class LoadQueuePeon
|
|||
SegmentHolder holder = new SegmentHolder(segment, DROP, Arrays.asList(callback));
|
||||
|
||||
synchronized (lock) {
|
||||
if (segmentsToDrop.contains(holder)) {
|
||||
final SegmentHolder existingHolder = segmentsToDrop.get(holder);
|
||||
if (existingHolder != null) {
|
||||
if (callback != null) {
|
||||
holder.addCallback(callback);
|
||||
existingHolder.addCallback(callback);
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier());
|
||||
segmentsToDrop.add(holder);
|
||||
segmentsToDrop.put(holder, holder);
|
||||
doNext();
|
||||
}
|
||||
|
||||
|
@ -222,10 +225,10 @@ public class LoadQueuePeon
|
|||
synchronized (lock) {
|
||||
if (currentlyProcessing == null) {
|
||||
if (!segmentsToDrop.isEmpty()) {
|
||||
currentlyProcessing = segmentsToDrop.first();
|
||||
currentlyProcessing = segmentsToDrop.firstKey();
|
||||
log.info("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
|
||||
} else if (!segmentsToLoad.isEmpty()) {
|
||||
currentlyProcessing = segmentsToLoad.first();
|
||||
currentlyProcessing = segmentsToLoad.firstKey();
|
||||
log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
|
||||
} else {
|
||||
return;
|
||||
|
@ -337,7 +340,7 @@ public class LoadQueuePeon
|
|||
default:
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
|
||||
callBackExecutor.execute(
|
||||
new Runnable()
|
||||
{
|
||||
|
@ -361,14 +364,14 @@ public class LoadQueuePeon
|
|||
}
|
||||
|
||||
if (!segmentsToDrop.isEmpty()) {
|
||||
for (SegmentHolder holder : segmentsToDrop) {
|
||||
for (SegmentHolder holder : segmentsToDrop.keySet()) {
|
||||
holder.executeCallbacks();
|
||||
}
|
||||
}
|
||||
segmentsToDrop.clear();
|
||||
|
||||
if (!segmentsToLoad.isEmpty()) {
|
||||
for (SegmentHolder holder : segmentsToLoad) {
|
||||
for (SegmentHolder holder : segmentsToLoad.keySet()) {
|
||||
holder.executeCallbacks();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue