Skip to content

Commit

Permalink
Fix queries for updated segments on SinkQuerySegmentWalker (#17157)
Browse files Browse the repository at this point in the history
Fix the logic for usage of segment descriptors from queries in SinkQuerySegmentWalker when there are upgraded segments as a result of concurrent replace.

Concurrent append and replace:
With the introduction of concurrent append and replace, for a given interval:

The same sink can correspond to a base segment V0_x0, and have multiple mappings to higher versions with distinct partition numbers such as V1_x1.... Vn_xn.
The initial segment allocation can happen on version V0, but there can be several allocations during the lifecycle of a task which can have different versions spanning from V0 to Vn.
Changes:
Maintain a new timeline of (An overshadowable holding a SegmentDescriptor)
Every segment allocation of version upgrade adds the latest segment descriptor to this timeline.
Iterate this timeline instead of the sinkTimeline to get the segment descriptors in getQueryRunnerForIntervals
Also maintain a mapping of the upgraded segment to its base segment.
When a sink is needed to process the query, find the base segment corresponding to a given descriptor, and then use the sinkTimeline to find its chunk.
  • Loading branch information
AmatyaAvadhanula authored Oct 9, 2024
1 parent a395368 commit 88d26e4
Show file tree
Hide file tree
Showing 4 changed files with 863 additions and 77 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,10 @@
import org.apache.druid.segment.realtime.sink.Sink;
import org.apache.druid.segment.realtime.sink.SinkSegmentReference;
import org.apache.druid.server.ResourceIdPopulatingQueryRunner;
import org.apache.druid.timeline.Overshadowable;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.IntegerPartitionChunk;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.utils.CloseableUtils;
import org.joda.time.Interval;
Expand All @@ -74,8 +76,6 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors;
Expand All @@ -90,20 +90,19 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker

private final String dataSource;

private final VersionedIntervalTimeline<String, Sink> sinkTimeline;
// Maintain a timeline of ids and Sinks for all the segments including the base and upgraded versions
private final VersionedIntervalTimeline<String, SinkHolder> upgradedSegmentsTimeline;
private final ObjectMapper objectMapper;
private final ServiceEmitter emitter;
private final QueryRunnerFactoryConglomerate conglomerate;
private final QueryProcessingPool queryProcessingPool;
private final Cache cache;
private final CacheConfig cacheConfig;
private final CachePopulatorStats cachePopulatorStats;
private final ConcurrentMap<SegmentDescriptor, SegmentDescriptor> newIdToBasePendingSegment
= new ConcurrentHashMap<>();

public SinkQuerySegmentWalker(
String dataSource,
VersionedIntervalTimeline<String, Sink> sinkTimeline,
VersionedIntervalTimeline<String, SinkHolder> upgradedSegmentsTimeline,
ObjectMapper objectMapper,
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
Expand All @@ -114,7 +113,7 @@ public SinkQuerySegmentWalker(
)
{
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
this.sinkTimeline = Preconditions.checkNotNull(sinkTimeline, "sinkTimeline");
this.upgradedSegmentsTimeline = upgradedSegmentsTimeline;
this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper");
this.emitter = Preconditions.checkNotNull(emitter, "emitter");
this.conglomerate = Preconditions.checkNotNull(conglomerate, "conglomerate");
Expand All @@ -133,7 +132,7 @@ public <T> QueryRunner<T> getQueryRunnerForIntervals(final Query<T> query, final
{
final Iterable<SegmentDescriptor> specs = FunctionalIterable
.create(intervals)
.transformCat(sinkTimeline::lookup)
.transformCat(upgradedSegmentsTimeline::lookup)
.transformCat(
holder -> FunctionalIterable
.create(holder.getObject())
Expand Down Expand Up @@ -196,9 +195,8 @@ public <T> QueryRunner<T> getQueryRunnerForSegments(final Query<T> query, final
final LinkedHashMap<SegmentDescriptor, List<QueryRunner<T>>> allRunners = new LinkedHashMap<>();

try {
for (final SegmentDescriptor newDescriptor : specs) {
final SegmentDescriptor descriptor = newIdToBasePendingSegment.getOrDefault(newDescriptor, newDescriptor);
final PartitionChunk<Sink> chunk = sinkTimeline.findChunk(
for (final SegmentDescriptor descriptor : specs) {
final PartitionChunk<SinkHolder> chunk = upgradedSegmentsTimeline.findChunk(
descriptor.getInterval(),
descriptor.getVersion(),
descriptor.getPartitionNumber()
Expand All @@ -212,7 +210,7 @@ public <T> QueryRunner<T> getQueryRunnerForSegments(final Query<T> query, final
continue;
}

final Sink theSink = chunk.getObject();
final Sink theSink = chunk.getObject().sink;
final SegmentId sinkSegmentId = theSink.getSegment().getId();
segmentIdMap.put(descriptor, sinkSegmentId);
final List<SinkSegmentReference> sinkSegmentReferences =
Expand Down Expand Up @@ -361,26 +359,48 @@ public <T> QueryRunner<T> getQueryRunnerForSegments(final Query<T> query, final
}
}

public void registerUpgradedPendingSegment(
SegmentIdWithShardSpec basePendingSegment,
SegmentIdWithShardSpec upgradedPendingSegment
)
/**
* Must be called when a segment is announced by a task.
* Either the base segment upon allocation or any upgraded version due to a concurrent replace
*/
public void registerUpgradedPendingSegment(SegmentIdWithShardSpec id, Sink sink)
{
newIdToBasePendingSegment.put(
upgradedPendingSegment.asSegmentId().toDescriptor(),
basePendingSegment.asSegmentId().toDescriptor()
final SegmentDescriptor upgradedDescriptor = id.asSegmentId().toDescriptor();
upgradedSegmentsTimeline.add(
upgradedDescriptor.getInterval(),
upgradedDescriptor.getVersion(),
IntegerPartitionChunk.make(
null,
null,
upgradedDescriptor.getPartitionNumber(),
new SinkHolder(upgradedDescriptor, sink)
)
);
}

@VisibleForTesting
String getDataSource()
/**
* Must be called when dropping sink from the sinkTimeline
* It is the responsibility of the caller to unregister all associated ids including the base id
*/
public void unregisterUpgradedPendingSegment(SegmentIdWithShardSpec id, Sink sink)
{
return dataSource;
final SegmentDescriptor upgradedDescriptor = id.asSegmentId().toDescriptor();
upgradedSegmentsTimeline.remove(
upgradedDescriptor.getInterval(),
upgradedDescriptor.getVersion(),
IntegerPartitionChunk.make(
null,
null,
upgradedDescriptor.getPartitionNumber(),
new SinkHolder(upgradedDescriptor, sink)
)
);
}

public VersionedIntervalTimeline<String, Sink> getSinkTimeline()
@VisibleForTesting
String getDataSource()
{
return sinkTimeline;
return dataSource;
}

public static String makeHydrantCacheIdentifier(final FireHydrant hydrant)
Expand All @@ -395,4 +415,46 @@ public static String makeHydrantCacheIdentifier(final SegmentId segmentId, final
// with subsegments (hydrants).
return segmentId + "_H" + hydrantNumber;
}

private static class SinkHolder implements Overshadowable<SinkHolder>
{
private final Sink sink;
private final SegmentDescriptor segmentDescriptor;

private SinkHolder(SegmentDescriptor segmentDescriptor, Sink sink)
{
this.segmentDescriptor = segmentDescriptor;
this.sink = sink;
}

@Override
public int getStartRootPartitionId()
{
return segmentDescriptor.getPartitionNumber();
}

@Override
public int getEndRootPartitionId()
{
return segmentDescriptor.getPartitionNumber() + 1;
}

@Override
public String getVersion()
{
return segmentDescriptor.getVersion();
}

@Override
public short getMinorVersion()
{
return 0;
}

@Override
public short getAtomicUpdateGroupSize()
{
return 1;
}
}
}
Loading

0 comments on commit 88d26e4

Please sign in to comment.