From 4dc7d688dc79a13a7f6b48172ce0413506e9997a Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 8 Sep 2023 16:55:41 +0530 Subject: [PATCH 01/37] Add metadata fetch --- .../apache/druid/msq/exec/ControllerImpl.java | 25 +++++- .../input/table/DataSegmentWithLocation.java | 75 +++++++++++++++++ .../input/table/RichSegmentDescriptor.java | 38 +++++++-- .../msq/input/table/TableInputSpecSlicer.java | 4 +- .../msq/util/MultiStageQueryContext.java | 12 +++ .../table/RichSegmentDescriptorTest.java | 12 ++- .../input/table/SegmentsInputSliceTest.java | 4 +- .../input/table/TableInputSpecSlicerTest.java | 84 ++++++++++++++----- .../client/coordinator/CoordinatorClient.java | 6 ++ .../coordinator/CoordinatorClientImpl.java | 33 ++++++++ .../coordinator/NoopCoordinatorClient.java | 7 ++ 11 files changed, 266 insertions(+), 34 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 20033480f106..61516ea37e04 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -39,6 +39,7 @@ import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.StringTuple; import org.apache.druid.data.input.impl.DimensionSchema; @@ -140,6 +141,7 @@ import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.input.stage.StageInputSpecSlicer; +import org.apache.druid.msq.input.table.DataSegmentWithLocation; import org.apache.druid.msq.input.table.TableInputSpec; import org.apache.druid.msq.input.table.TableInputSpecSlicer; import org.apache.druid.msq.kernel.GlobalSortTargetSizeShuffleSpec; @@ -1158,14 +1160,31 @@ private QueryKit makeQueryControllerToolKit() private DataSegmentTimelineView makeDataSegmentTimelineView() { + final boolean isIncludeRealtime = MultiStageQueryContext.isIncludeRealtime(task.getQuerySpec().getQuery().context()); + return (dataSource, intervals) -> { - final Collection dataSegments = + final Iterable serverViewSegments; + + // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two calls. + if (isIncludeRealtime) { + serverViewSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); + } else { + serverViewSegments = ImmutableList.of(); + } + + final Collection metadataStoreSegments = FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true); - if (dataSegments.isEmpty()) { + Set unifiedSegmentView = new HashSet<>(metadataStoreSegments); + for (ImmutableSegmentLoadInfo segmentLoadInfo : serverViewSegments) { + DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation(segmentLoadInfo.getSegment(), segmentLoadInfo.getServers()); + unifiedSegmentView.add(dataSegmentWithLocation); + } + + if (unifiedSegmentView.isEmpty()) { return Optional.empty(); } else { - return Optional.of(SegmentTimeline.forSegments(dataSegments)); + return Optional.of(SegmentTimeline.forSegments(unifiedSegmentView)); } }; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java new file mode 100644 index 000000000000..6ff037eedf63 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -0,0 +1,75 @@ +package org.apache.druid.msq.input.table; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.druid.jackson.CommaListJoinDeserializer; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Data segment including the locations which contain the segment. Used if MSQ needs to fetch the segment from a server + * instead of from deep storage. + */ +public class DataSegmentWithLocation extends DataSegment +{ + private final Set servers; + + @JsonCreator + public DataSegmentWithLocation( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution + @JsonProperty("loadSpec") @Nullable Map loadSpec, + @JsonProperty("dimensions") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List dimensions, + @JsonProperty("metrics") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List metrics, + @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, + @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, + @JsonProperty("binaryVersion") Integer binaryVersion, + @JsonProperty("size") long size, + @JsonProperty("servers") Set servers, + @JacksonInject PruneSpecsHolder pruneSpecsHolder + ) + { + super(dataSource, interval, version, loadSpec, dimensions, metrics, shardSpec, lastCompactionState, binaryVersion, size, pruneSpecsHolder); + this.servers = servers; + } + + public DataSegmentWithLocation( + DataSegment dataSegment, + Set servers + ) + { + super(dataSegment.getDataSource(), dataSegment.getInterval(), dataSegment.getVersion(), dataSegment.getLoadSpec(), dataSegment.getDimensions(), dataSegment.getMetrics(), dataSegment.getShardSpec(), dataSegment.getBinaryVersion(), dataSegment.getSize()); + this.servers = servers; + } + + @JsonProperty + public Set getServers() + { + return servers; + } + + @JsonIgnore + public boolean isRealtime() + { + return getLoadSpec() == null || getLoadSpec().isEmpty(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index 3ca48ef9cbdf..f482bdc581d9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -23,19 +23,22 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.server.coordination.DruidServerMetadata; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Objects; +import java.util.Set; /** * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment. * (SegmentDescriptor only provides the clipped interval.) - * + *
* To keep the serialized form lightweight, the full interval is only serialized if it is different from the * clipped interval. - * + *
* It is possible to deserialize this class as {@link SegmentDescriptor}. However, going the other direction is * not a good idea, because the {@link #fullInterval} will not end up being set correctly. */ @@ -43,16 +46,22 @@ public class RichSegmentDescriptor extends SegmentDescriptor { @Nullable private final Interval fullInterval; + private final Set servers; + private final Boolean isRealtime; public RichSegmentDescriptor( final Interval fullInterval, final Interval interval, final String version, - final int partitionNumber + final int partitionNumber, + final Set servers, + final boolean isRealtime ) { super(interval, version, partitionNumber); this.fullInterval = interval.equals(Preconditions.checkNotNull(fullInterval, "fullInterval")) ? null : fullInterval; + this.servers = servers; + this.isRealtime = isRealtime; } @JsonCreator @@ -60,17 +69,36 @@ static RichSegmentDescriptor fromJson( @JsonProperty("fi") @Nullable final Interval fullInterval, @JsonProperty("itvl") final Interval interval, @JsonProperty("ver") final String version, - @JsonProperty("part") final int partitionNumber + @JsonProperty("part") final int partitionNumber, + @JsonProperty("servers") final Set servers, + @JsonProperty("isRealtime") final Boolean isRealtime ) { return new RichSegmentDescriptor( fullInterval != null ? fullInterval : interval, interval, version, - partitionNumber + partitionNumber, + servers == null ? ImmutableSet.of() : servers, + isRealtime != null && isRealtime ); } + @JsonProperty("servers") + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + public Set getServers() + { + return servers; + } + + @JsonProperty("isRealtime") + @JsonInclude(JsonInclude.Include.NON_NULL) + public boolean isRealtime() + { + return Boolean.TRUE.equals(isRealtime); + } + public Interval getFullInterval() { return fullInterval == null ? getInterval() : fullInterval; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 37a97d33be5c..ba2300a0d7a1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -166,7 +166,9 @@ public RichSegmentDescriptor toRichSegmentDescriptor() segment.getInterval(), interval, segment.getVersion(), - segment.getShardSpec().getPartitionNum() + segment.getShardSpec().getPartitionNum(), + segment instanceof DataSegmentWithLocation ? ((DataSegmentWithLocation) segment).getServers() : null, + segment instanceof DataSegmentWithLocation && ((DataSegmentWithLocation) segment).isRealtime() ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 265f5eae0fe1..6273aa0bd4b4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -90,6 +90,9 @@ public class MultiStageQueryContext public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; + public static final String CTX_INCLUDE_REALTIME = "includeRealtimeResults"; + public static final boolean DEFAULT_INCLUDE_REALTIME = false; + public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_SELECT_DESTINATION = "selectDestination"; @@ -181,6 +184,15 @@ public static boolean isFinalizeAggregations(final QueryContext queryContext) ); } + public static boolean isIncludeRealtime(final QueryContext queryContext) + { + return QueryContexts.getAsBoolean( + CTX_INCLUDE_REALTIME, + queryContext.getBoolean(CTX_INCLUDE_REALTIME, DEFAULT_INCLUDE_REALTIME), + DEFAULT_INCLUDE_REALTIME + ); + } + public static WorkerAssignmentStrategy getAssignmentStrategy(final QueryContext queryContext) { return QueryContexts.getAsEnum( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java index 8884a92a665b..a476416a358d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java @@ -37,7 +37,9 @@ public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception Intervals.of("2000/2002"), Intervals.of("2000/2001"), "2", - 3 + 3, + null, + false ); Assert.assertEquals( @@ -54,7 +56,9 @@ public void testSerdeWithFullIntervalSameAsInterval() throws Exception Intervals.of("2000/2001"), Intervals.of("2000/2001"), "2", - 3 + 3, + null, + false ); Assert.assertEquals( @@ -71,7 +75,9 @@ public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exc Intervals.of("2000/2002"), Intervals.of("2000/2001"), "2", - 3 + 3, + null, + false ); Assert.assertEquals( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index df2937f30036..d038a6976d10 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -44,7 +44,9 @@ public void testSerde() throws Exception Intervals.of("2000/P1M"), Intervals.of("2000/P1M"), "1", - 0 + 0, + null, + false ) ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java index dbcb3646e887..d4660ebde79a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java @@ -146,25 +146,33 @@ public void test_sliceStatic_intervalFilter() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT1.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -206,7 +214,9 @@ public void test_sliceStatic_dimFilter() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -237,7 +247,9 @@ public void test_sliceStatic_intervalAndDimFilter() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -248,7 +260,9 @@ public void test_sliceStatic_intervalAndDimFilter() SEGMENT1.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -270,13 +284,17 @@ public void test_sliceStatic_oneSlice() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -298,7 +316,9 @@ public void test_sliceStatic_needTwoSlices() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -309,7 +329,9 @@ public void test_sliceStatic_needTwoSlices() SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -331,7 +353,9 @@ public void test_sliceStatic_threeSlices() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -342,7 +366,9 @@ public void test_sliceStatic_threeSlices() SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -385,13 +411,17 @@ public void test_sliceDynamic_maxOneSlice() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -418,13 +448,17 @@ public void test_sliceDynamic_needOne() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -451,7 +485,9 @@ public void test_sliceDynamic_needTwoDueToFiles() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -462,7 +498,9 @@ public void test_sliceDynamic_needTwoDueToFiles() SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) @@ -489,7 +527,9 @@ public void test_sliceDynamic_needTwoDueToBytes() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum() + SEGMENT1.getShardSpec().getPartitionNum(), + null, + false ) ) ), @@ -500,7 +540,9 @@ public void test_sliceDynamic_needTwoDueToBytes() SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum() + SEGMENT2.getShardSpec().getPartitionNum(), + null, + false ) ) ) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index 08110f61f059..336576b675dc 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -20,6 +20,7 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.timeline.DataSegment; @@ -40,6 +41,11 @@ public interface CoordinatorClient */ ListenableFuture fetchSegment(String dataSource, String segmentId, boolean includeUnused); + /** + * Fetches segments from the coordinator server view for the given dataSource and intervals. + */ + Iterable fetchServerViewSegments(String dataSource, List intervals); + /** * Fetches segment metadata for the given dataSource and intervals. */ diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index e93cbe830b3f..aa204508f5e5 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -34,6 +35,7 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; +import java.util.ArrayList; import java.util.List; public class CoordinatorClientImpl implements CoordinatorClient @@ -89,6 +91,37 @@ public ListenableFuture fetchSegment(String dataSource, String segm ); } + @Override + public Iterable fetchServerViewSegments(String dataSource, List intervals) + { + ArrayList retVal = new ArrayList<>(); + for (Interval interval : intervals) { + String intervalString = interval.toString().replace("/", "_"); + + final String path = StringUtils.format( + "/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?full", + StringUtils.urlEncode(dataSource), + intervalString + ); + ListenableFuture> segments = FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + new TypeReference>() + { + } + ) + ); + FutureUtils.getUnchecked(segments, true).forEach(retVal::add); // TODO: reconsider + } + + return retVal; + } + @Override public ListenableFuture> fetchUsedSegments(String dataSource, List intervals) { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 76e6346d3808..1bc23b48a478 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -20,6 +20,7 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.timeline.DataSegment; @@ -41,6 +42,12 @@ public ListenableFuture fetchSegment(String dataSource, String segm throw new UnsupportedOperationException(); } + @Override + public Iterable fetchServerViewSegments(String dataSource, List intervals) + { + throw new UnsupportedOperationException(); + } + @Override public ListenableFuture> fetchUsedSegments(String dataSource, List intervals) { From c2289d132457faa05fc35b769df7c6df42e38ee9 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 22 Sep 2023 10:21:42 +0530 Subject: [PATCH 02/37] Add ability to query data servers --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../msq/exec/TaskDataSegmentProvider.java | 57 +++++- .../msq/indexing/IndexerWorkerContext.java | 8 +- .../external/ExternalInputSliceReader.java | 3 + .../input/inline/InlineInputSliceReader.java | 3 + .../input/lookup/LookupInputSliceReader.java | 3 + .../input/table/DataSegmentWithLocation.java | 19 ++ .../input/table/SegmentWithDescriptor.java | 27 ++- .../input/table/SegmentsInputSliceReader.java | 1 + .../msq/querykit/BaseLeafFrameProcessor.java | 8 +- .../msq/querykit/DataSegmentProvider.java | 16 +- .../GroupByPreShuffleFrameProcessor.java | 31 ++++ .../scan/ScanQueryFrameProcessor.java | 52 ++++++ .../msq/exec/TaskDataSegmentProviderTest.java | 6 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 28 ++- .../apache/druid/msq/test/MSQTestBase.java | 28 ++- .../coordinator/CoordinatorClientImpl.java | 2 +- .../druid/discovery/DataServerClient.java | 173 ++++++++++++++++++ .../druid/rpc/FixedSetServiceLocator.java | 73 ++++++++ .../org/apache/druid/rpc/ServiceLocation.java | 36 ++++ 20 files changed, 562 insertions(+), 14 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/discovery/DataServerClient.java create mode 100644 server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 61516ea37e04..1921aa75e2d9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1167,7 +1167,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two calls. if (isIncludeRealtime) { - serverViewSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); + serverViewSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); } else { serverViewSegments = ImmutableList.of(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java index c327ec340aea..b934a576b46f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -19,16 +19,27 @@ package org.apache.druid.msq.exec; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.Queries; +import org.apache.druid.query.Query; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.context.DefaultResponseContext; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -45,6 +56,7 @@ import java.io.IOException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.function.Function; import java.util.function.Supplier; /** @@ -55,17 +67,23 @@ public class TaskDataSegmentProvider implements DataSegmentProvider private final CoordinatorClient coordinatorClient; private final SegmentCacheManager segmentCacheManager; private final IndexIO indexIO; + private final ServiceClientFactory serviceClientFactory; + private final ObjectMapper objectMapper; private final ConcurrentHashMap holders; public TaskDataSegmentProvider( CoordinatorClient coordinatorClient, SegmentCacheManager segmentCacheManager, - IndexIO indexIO + IndexIO indexIO, + ServiceClientFactory serviceClientFactory, + ObjectMapper objectMapper ) { this.coordinatorClient = coordinatorClient; this.segmentCacheManager = segmentCacheManager; this.indexIO = indexIO; + this.serviceClientFactory = serviceClientFactory; + this.objectMapper = objectMapper; this.holders = new ConcurrentHashMap<>(); } @@ -143,6 +161,43 @@ private ReferenceCountingResourceHolder fetchSegmentInternal( } } + @Override + public Function, ResourceHolder>> fetchLoadedSegment( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters + ) + { + return query -> fetchServedSegmentInternal( + segmentDescriptor, + dataSource, + channelCounters, + query + ); + } + + private ResourceHolder> fetchServedSegmentInternal( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters, + Query query + ) + { + query = query.withDataSource(new TableDataSource(dataSource)); + query = Queries.withSpecificSegments( + query, + ImmutableList.of(segmentDescriptor) + ); + + DataServerClient dataServerClient = new DataServerClient<>( + serviceClientFactory, + new FixedSetServiceLocator(segmentDescriptor.getServers()), + objectMapper + ); + DefaultResponseContext responseContext = new DefaultResponseContext(); + return dataServerClient.run(query, responseContext); + } + private static class SegmentHolder implements Supplier> { private final Supplier> holderSupplier; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 43d067dd6c90..f1d69212f4c8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -104,7 +104,13 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to toolbox, injector, indexIO, - new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO), + new TaskDataSegmentProvider( + toolbox.getCoordinatorClient(), + segmentCacheManager, + indexIO, + serviceClientFactory, + toolbox.getJsonMapper() + ), serviceClientFactory ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 3dbd3da0a026..0ca478cd4dae 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -159,6 +159,9 @@ private static Iterator inputSourceSegmentIterator( ); return new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), + query -> { + throw new AssertionError("Should not be called"); + }, segmentId.toDescriptor() ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 143fb49692f2..e6b1489dfaac 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -74,6 +74,9 @@ public ReadableInputs attach( segment -> ReadableInput.segment( new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), + query -> { + throw new AssertionError("Should not be called"); + }, DUMMY_SEGMENT_DESCRIPTOR ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index 648527ce0061..a0e8253c88cf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -99,6 +99,9 @@ public ReadableInputs attach( return ResourceHolder.fromCloseable(segment); }, + query -> { + throw new AssertionError("Should not be called"); + }, SegmentId.dummy(lookupName).toDescriptor() ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java index 6ff037eedf63..df937611309c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.msq.input.table; import com.fasterxml.jackson.annotation.JacksonInject; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 020b9f2a5bb0..3d06199a9805 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -21,10 +21,13 @@ import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; import java.util.Objects; +import java.util.function.Function; import java.util.function.Supplier; /** @@ -33,30 +36,35 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; + private final Function, ResourceHolder>> servedSegmentRowsFunction; private final SegmentDescriptor descriptor; /** * Create a new instance. * - * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} logic - * must include a delegated call to {@link Segment#close()}. - * @param descriptor segment descriptor + * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} + * logic must include a delegated call to {@link Segment#close()}. + * @param servedSegmentRowsFunction function which accepts a {@link Query} and fetches a {@link Sequence} of results + * from a server where the segment is served. + * @param descriptor segment descriptor */ public SegmentWithDescriptor( final Supplier> segmentSupplier, + final Function, ResourceHolder>> servedSegmentRowsFunction, final SegmentDescriptor descriptor ) { this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment"); + this.servedSegmentRowsFunction = servedSegmentRowsFunction; this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor"); } /** * The physical segment. - * + *
* Named "getOrLoad" because the segment may be generated by a lazy supplier. In this case, the segment is acquired * as part of the call to this method. - * + *
* It is not necessary to call {@link Segment#close()} on the returned segment. Calling {@link ResourceHolder#close()} * is enough. */ @@ -65,6 +73,10 @@ public ResourceHolder getOrLoad() return segmentSupplier.get(); } + public ResourceHolder getServedSegmentFromServer(Query query) + { + return servedSegmentRowsFunction.apply(query); + } /** * The segment descriptor associated with this physical segment. */ @@ -73,6 +85,11 @@ public SegmentDescriptor getDescriptor() return descriptor; } + public boolean isRealtimeSegment() + { + return descriptor instanceof RichSegmentDescriptor && ((RichSegmentDescriptor) descriptor).isRealtime(); + } + @Override public boolean equals(Object o) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index 5334c4cb2abd..d310d4af3d37 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -94,6 +94,7 @@ private Iterator dataSegmentIterator( return new SegmentWithDescriptor( dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), + dataSegmentProvider.fetchLoadedSegment(descriptor, dataSource, channelCounters), descriptor ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index d5b31328b0b8..1ea87f89db0b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -156,7 +156,12 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws } else if (readableInputs.size() != inputChannels.size()) { return ReturnOrAwait.awaitAll(inputChannels.size()); } else if (baseInput.hasSegment()) { - return runWithSegment(baseInput.getSegment()); + SegmentWithDescriptor segment = baseInput.getSegment(); + if (segment.isRealtimeSegment()) { + return runWithLoadedSegment(baseInput.getSegment()); + } else { + return runWithSegment(baseInput.getSegment()); + } } else { assert baseInput.hasChannel(); return runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); @@ -175,6 +180,7 @@ protected FrameWriterFactory getFrameWriterFactory() } protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; + protected abstract ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException; protected abstract ReturnOrAwait runWithInputChannel( ReadableFrameChannel inputChannel, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java index 0e931c7f8ef0..5bf9f4f2793f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java @@ -20,10 +20,14 @@ package org.apache.druid.msq.querykit; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.query.Query; import org.apache.druid.segment.Segment; import org.apache.druid.timeline.SegmentId; +import java.util.function.Function; import java.util.function.Supplier; public interface DataSegmentProvider @@ -32,7 +36,7 @@ public interface DataSegmentProvider * Returns a supplier that fetches the segment corresponding to the provided segmentId from deep storage. The segment * is not actually fetched until you call {@link Supplier#get()}. Once you call this, make sure to also call * {@link ResourceHolder#close()}. - * + *
* It is not necessary to call {@link ResourceHolder#close()} if you never call {@link Supplier#get()}. */ Supplier> fetchSegment( @@ -40,4 +44,14 @@ Supplier> fetchSegment( ChannelCounters channelCounters, boolean isReindex ); + + /** + * Returns a function that fetches the segment corresponding to the provided segmentId from a server where it is + * loaded. The segment is not actually fetched until you call {@link Function#apply(Object)}. + */ + Function, ResourceHolder>> fetchLoadedSegment( + RichSegmentDescriptor richSegmentDescriptor, + String dataSource, + ChannelCounters channelCounters + ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 67e53cd41e7d..ad742c9c21ed 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit.groupby; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.collections.ResourceHolder; @@ -42,6 +43,7 @@ import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; @@ -97,6 +99,35 @@ public GroupByPreShuffleFrameProcessor( ); } + private GroupByQuery prepareGroupByQuery(GroupByQuery query) + { + ImmutableMap overridingContext = ImmutableMap.builder() + .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, query.isApplyLimitPushDown()) + .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) + .build(); + return query.withOverriddenContext(overridingContext); + } + + @Override + protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException + { + if (resultYielder == null) { + ResourceHolder servedSegmentFromServer = closer.register( + segment.getServedSegmentFromServer(prepareGroupByQuery(query)) + ); + Sequence rowSequence = servedSegmentFromServer.get(); + resultYielder = Yielders.each(rowSequence); + } + + populateFrameWriterAndFlushIfNeeded(); + + if (resultYielder == null || resultYielder.isDone()) { + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.runAgain(); + } + } + @Override protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 543dd01f5b6b..6880e33de2e1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntSet; @@ -43,6 +44,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; @@ -52,18 +54,22 @@ import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.RowBasedCursor; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -82,6 +88,7 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor { private final ScanQuery query; private final AtomicLong runningCountForLimit; + private final ObjectMapper jsonMapper; private final SettableLongVirtualColumn partitionBoostVirtualColumn; private final VirtualColumns frameWriterVirtualColumns; private final Closer closer = Closer.create(); @@ -114,6 +121,7 @@ public ScanQueryFrameProcessor( ); this.query = query; this.runningCountForLimit = runningCountForLimit; + this.jsonMapper = jsonMapper; this.partitionBoostVirtualColumn = new SettableLongVirtualColumn(QueryKitUtils.PARTITION_BOOST_COLUMN); final List frameWriterVirtualColumns = new ArrayList<>(); @@ -154,6 +162,50 @@ public void cleanup() throws IOException closer.close(); } + @Override + protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException + { + if (cursor == null) { + final ResourceHolder resourceHolder = closer.register(segment.getServedSegmentFromServer(query)); + Sequence sequence = resourceHolder.get(); + + Sequence parsedSequence = sequence.flatMap(resultRow -> { + ScanResultValue scanResultValue = resultRow; + List> events = (List>) scanResultValue.getEvents(); + return Sequences.simple(events); + }).map(List::toArray); + RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); + RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromSequence( + parsedSequence, + rowSignature + ); + + final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable))); + + if (cursorYielder.isDone()) { + // No cursors! + cursorYielder.close(); + return ReturnOrAwait.returnObject(rowsOutput); + } else { + final long rowsFlushed = setNextCursor(cursorYielder.get(), null); + assert rowsFlushed == 0; // There's only ever one cursor when running with a segment + closer.register(cursorYielder); + } + } + + populateFrameWriterAndFlushIfNeededWithExceptionHandling(); + + if (cursor.isDone()) { + flushFrameWriter(); + } + + if (cursor.isDone() && (frameWriter == null || frameWriter.getNumRows() == 0)) { + return ReturnOrAwait.returnObject(rowsOutput); + } else { + return ReturnOrAwait.runAgain(); + } + } + @Override protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index eafb443bafc1..067acc94957d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -69,6 +70,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.File; @@ -150,7 +152,9 @@ public void setUp() throws Exception provider = new TaskDataSegmentProvider( new TestCoordinatorClientImpl(), cacheManager, - indexIO + indexIO, + Mockito.mock(ServiceClientFactory.class), + jsonMapper ); exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(THREADS, getClass().getSimpleName() + "-%s")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 0c840b14a88e..1ae16dd94d9d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -38,13 +38,17 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.ForwardingQueryProcessingPool; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -84,6 +88,7 @@ import java.io.IOException; import java.util.List; import java.util.Set; +import java.util.function.Function; import java.util.function.Supplier; import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE1; @@ -161,7 +166,28 @@ public String getFormatString() )); binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bind(DataSegmentProvider.class) - .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); + .toInstance(new DataSegmentProvider() + { + @Override + public Supplier> fetchSegment( + SegmentId segmentId, + ChannelCounters channelCounters, + boolean isReindex + ) + { + return getSupplierForSegment(segmentId); + } + + @Override + public Function, ResourceHolder>> fetchLoadedSegment( + RichSegmentDescriptor richSegmentDescriptor, + String dataSource, + ChannelCounters channelCounters + ) + { + throw new AssertionError(); + } + }); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 6b8ea3e0ae0b..3fbe849259cf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -71,12 +71,14 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.input.InputSourceModule; +import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; @@ -104,6 +106,7 @@ import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; @@ -113,6 +116,7 @@ import org.apache.druid.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.ForwardingQueryProcessingPool; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -205,6 +209,7 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -414,7 +419,28 @@ public String getFormatString() binder.bind(QueryProcessingPool.class) .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(DataSegmentProvider.class) - .toInstance((dataSegment, channelCounters, isReindex) -> getSupplierForSegment(dataSegment)); + .toInstance(new DataSegmentProvider() + { + @Override + public Supplier> fetchSegment( + SegmentId segmentId, + ChannelCounters channelCounters, + boolean isReindex + ) + { + return getSupplierForSegment(segmentId); + } + + @Override + public Function, ResourceHolder>> fetchLoadedSegment( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters + ) + { + throw new AssertionError(); + } + }); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index aa204508f5e5..eb0af158a15c 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -96,7 +96,7 @@ public Iterable fetchServerViewSegments(String dataSou { ArrayList retVal = new ArrayList<>(); for (Interval interval : intervals) { - String intervalString = interval.toString().replace("/", "_"); + String intervalString = StringUtils.replace(interval.toString(), "/", "_"); final String path = StringUtils.format( "/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?full", diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java new file mode 100644 index 000000000000..d4fa6c2fbf75 --- /dev/null +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.discovery; + +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.JsonParserIterator; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.query.Query; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.utils.CloseableUtils; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import java.io.InputStream; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Client to query data servers given a query. + */ +public class DataServerClient +{ + private static final Logger log = new Logger(DataServerClient.class); + private static final String SERVED_SEGMENT_CLIENT_NAME = "ServedSegmentClient"; + private final ServiceClient serviceClient; + private final ObjectMapper objectMapper; + private final ScheduledExecutorService queryCancellationExecutor; + + public DataServerClient( + ServiceClientFactory serviceClientFactory, + FixedSetServiceLocator fixedSetServiceLocator, + ObjectMapper objectMapper + ) + { + serviceClient = serviceClientFactory.makeClient( + SERVED_SEGMENT_CLIENT_NAME, + fixedSetServiceLocator, + StandardRetryPolicy.aboutAnHour() + ); + this.objectMapper = objectMapper; + this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); + } + + + public ResourceHolder> run(Query query, ResponseContext context) + { + final String basePath = "/druid/v2/"; + final String cancelPath = basePath + query.getId(); + JavaType queryResultType; + if (query instanceof ScanQuery) { + queryResultType = objectMapper.getTypeFactory().constructType(ScanResultValue.class); + } else { + queryResultType = objectMapper.getTypeFactory().constructType(ResultRow.class); + } + + Future resultStreamFuture; + try { + resultStreamFuture = serviceClient.asyncRequest( + new RequestBuilder(HttpMethod.POST, basePath) + .jsonContent(objectMapper, query), // TODO: smile + new InputStreamResponseHandler() + ); + + BaseSequence> baseSequence = new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public JsonParserIterator make() + { + return new JsonParserIterator<>( + queryResultType, + resultStreamFuture, + basePath, + query, + "", + objectMapper + ); + } + + @Override + public void cleanup(JsonParserIterator iterFromMake) + { + CloseableUtils.closeAndWrapExceptions(iterFromMake); + } + } + ); + + return new ResourceHolder>() + { + @Override + public Sequence get() + { + return baseSequence; + } + + @Override + public void close() + { + if (!resultStreamFuture.isDone()) { + cancelQuery(query, cancelPath); + } + } + }; + } + catch (Exception e) { + cancelQuery(query, cancelPath); + throw new RuntimeException(e); + } + } + + private void cancelQuery(Query query, String cancelPath) + { + Runnable cancelRunnable = () -> { + Future cancelFuture = serviceClient.asyncRequest( + new RequestBuilder(HttpMethod.DELETE, cancelPath), + StatusResponseHandler.getInstance()); + + Runnable checkRunnable = () -> { + try { + if (!cancelFuture.isDone()) { + log.error("Error cancelling query[%s]", query); + } + StatusResponseHolder response = cancelFuture.get(); + if (response.getStatus().getCode() >= 500) { + log.error("Error cancelling query[%s]: queryable node returned status[%d] [%s].", + query, + response.getStatus().getCode(), + response.getStatus().getReasonPhrase()); + } + } + catch (ExecutionException | InterruptedException e) { + log.error(e, "Error cancelling query[%s]", query); + } + }; + queryCancellationExecutor.schedule(checkRunnable, 5, TimeUnit.SECONDS); + }; + queryCancellationExecutor.submit(cancelRunnable); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java new file mode 100644 index 000000000000..fb6d0075c39d --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.rpc; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.jboss.netty.util.internal.ThreadLocalRandom; + +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Basic implmentation of {@link ServiceLocator} that returns a service location from a static set of locations. Returns + * a random location each time one is requested. + */ +public class FixedSetServiceLocator implements ServiceLocator +{ + private ServiceLocations serviceLocations; + + public FixedSetServiceLocator(Set servers) + { + if (servers == null) { + serviceLocations = ServiceLocations.closed(); + } else { + Set serviceLocationSet = servers.stream() + .map(ServiceLocation::fromDruidServerMetadata) + .collect(Collectors.toSet()); + serviceLocations = ServiceLocations.forLocations(serviceLocationSet); + } + } + + @Override + public ListenableFuture locate() + { + if (serviceLocations.isClosed() || serviceLocations.getLocations().isEmpty()) { + return Futures.immediateFuture(ServiceLocations.closed()); + } + + Set locationSet = serviceLocations.getLocations(); + return Futures.immediateFuture( + ServiceLocations.forLocation( + locationSet.stream() + .skip(new ThreadLocalRandom().nextInt(locationSet.size())) + .findFirst() + .orElse(null) + ) + ); + } + + @Override + public void close() + { + serviceLocations = ServiceLocations.closed(); + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java index eab82df328a1..a43e3727271b 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -20,8 +20,13 @@ package org.apache.druid.rpc; import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DruidServerMetadata; +import javax.annotation.Nullable; +import java.util.Iterator; import java.util.Objects; /** @@ -47,6 +52,37 @@ public static ServiceLocation fromDruidNode(final DruidNode druidNode) return new ServiceLocation(druidNode.getHost(), druidNode.getPlaintextPort(), druidNode.getTlsPort(), ""); } + private static final Splitter SPLITTER = Splitter.on(":").limit(2); + + public static ServiceLocation fromDruidServerMetadata(final DruidServerMetadata druidServerMetadata) + { + final String host = getHostFromString(druidServerMetadata.getHostAndPort()); + int plaintextPort = getPortFromString(druidServerMetadata.getHostAndPort()); + int tlsPort = getPortFromString(druidServerMetadata.getHostAndTlsPort()); + return new ServiceLocation(host, plaintextPort, tlsPort, ""); + } + + @Nullable + private static String getHostFromString(String s) + { + if (s == null) { + return null; + } + Iterator iterator = SPLITTER.split(s).iterator(); + ImmutableList strings = ImmutableList.copyOf(iterator); + return strings.get(0); + } + + private static int getPortFromString(String s) + { + if (s == null) { + return -1; + } + Iterator iterator = SPLITTER.split(s).iterator(); + ImmutableList strings = ImmutableList.copyOf(iterator); + return Integer.parseInt(strings.get(1)); + } + public String getHost() { return host; From 5296d1d380b5f3c6edd9aaeb049de822c5935254 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 22 Sep 2023 10:49:25 +0530 Subject: [PATCH 03/37] Cleanup --- .../apache/druid/client/coordinator/CoordinatorClientImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index eb0af158a15c..f82beb2778d1 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -116,7 +116,7 @@ public Iterable fetchServerViewSegments(String dataSou } ) ); - FutureUtils.getUnchecked(segments, true).forEach(retVal::add); // TODO: reconsider + FutureUtils.getUnchecked(segments, true).forEach(retVal::add); } return retVal; From 80da7b2a929f61351ec1f396ad1abda98dc136ef Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 24 Sep 2023 15:35:35 +0530 Subject: [PATCH 04/37] Add smile format --- .../druid/msq/exec/TaskDataSegmentProvider.java | 8 ++++++-- .../msq/indexing/IndexerWorkerContext.java | 5 ++++- .../msq/exec/TaskDataSegmentProviderTest.java | 1 + .../druid/discovery/DataServerClient.java | 17 ++++++++++------- 4 files changed, 21 insertions(+), 10 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java index b934a576b46f..cbe08095128f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -70,13 +70,15 @@ public class TaskDataSegmentProvider implements DataSegmentProvider private final ServiceClientFactory serviceClientFactory; private final ObjectMapper objectMapper; private final ConcurrentHashMap holders; + private final ObjectMapper smileMapper; public TaskDataSegmentProvider( CoordinatorClient coordinatorClient, SegmentCacheManager segmentCacheManager, IndexIO indexIO, ServiceClientFactory serviceClientFactory, - ObjectMapper objectMapper + ObjectMapper objectMapper, + ObjectMapper smileMapper ) { this.coordinatorClient = coordinatorClient; @@ -85,6 +87,7 @@ public TaskDataSegmentProvider( this.serviceClientFactory = serviceClientFactory; this.objectMapper = objectMapper; this.holders = new ConcurrentHashMap<>(); + this.smileMapper = smileMapper; } @Override @@ -192,7 +195,8 @@ private ResourceHolder> fetchServedSegmentInternal( DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, new FixedSetServiceLocator(segmentDescriptor.getServers()), - objectMapper + objectMapper, + smileMapper ); DefaultResponseContext responseContext = new DefaultResponseContext(); return dataServerClient.run(query, responseContext); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index f1d69212f4c8..386a1f99af88 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -27,6 +27,7 @@ import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.concurrent.Execs; @@ -99,6 +100,7 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch")); final ServiceClientFactory serviceClientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); + final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class)); return new IndexerWorkerContext( toolbox, @@ -109,7 +111,8 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to segmentCacheManager, indexIO, serviceClientFactory, - toolbox.getJsonMapper() + toolbox.getJsonMapper(), + smileMapper ), serviceClientFactory ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index 067acc94957d..7c1fa40cce39 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -154,6 +154,7 @@ public void setUp() throws Exception cacheManager, indexIO, Mockito.mock(ServiceClientFactory.class), + jsonMapper, jsonMapper ); diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index d4fa6c2fbf75..164ea97aa635 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -21,13 +21,13 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.query.Query; @@ -58,12 +58,14 @@ public class DataServerClient private static final String SERVED_SEGMENT_CLIENT_NAME = "ServedSegmentClient"; private final ServiceClient serviceClient; private final ObjectMapper objectMapper; + private final ObjectMapper smileMapper; private final ScheduledExecutorService queryCancellationExecutor; public DataServerClient( ServiceClientFactory serviceClientFactory, FixedSetServiceLocator fixedSetServiceLocator, - ObjectMapper objectMapper + ObjectMapper objectMapper, + ObjectMapper smileMapper ) { serviceClient = serviceClientFactory.makeClient( @@ -72,6 +74,7 @@ public DataServerClient( StandardRetryPolicy.aboutAnHour() ); this.objectMapper = objectMapper; + this.smileMapper = smileMapper; this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } @@ -89,11 +92,11 @@ public ResourceHolder> run(Query query, ResponseContext context) Future resultStreamFuture; try { - resultStreamFuture = serviceClient.asyncRequest( + resultStreamFuture = Futures.immediateFuture(serviceClient.request( new RequestBuilder(HttpMethod.POST, basePath) - .jsonContent(objectMapper, query), // TODO: smile - new InputStreamResponseHandler() - ); + .smileContent(smileMapper, query), + new DataServerResponseHandler(query, context, smileMapper) + )); BaseSequence> baseSequence = new BaseSequence<>( new BaseSequence.IteratorMaker>() @@ -107,7 +110,7 @@ public JsonParserIterator make() basePath, query, "", - objectMapper + smileMapper ); } From e2fe88366e37b92b33b7bf401591473c60e57716 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 24 Sep 2023 19:32:05 +0530 Subject: [PATCH 05/37] Clean --- .../msq/exec/TaskDataSegmentProvider.java | 28 +++-- .../external/ExternalInputSliceReader.java | 2 +- .../input/inline/InlineInputSliceReader.java | 2 +- .../input/lookup/LookupInputSliceReader.java | 2 +- .../input/table/SegmentWithDescriptor.java | 11 +- .../msq/querykit/DataSegmentProvider.java | 4 +- .../GroupByPreShuffleFrameProcessor.java | 9 +- .../scan/ScanQueryFrameProcessor.java | 21 ++-- .../druid/msq/test/CalciteMSQTestsHelper.java | 3 +- .../apache/druid/msq/test/MSQTestBase.java | 3 +- .../druid/discovery/DataServerClient.java | 2 +- .../discovery/DataServerResponseHandler.java | 108 ++++++++++++++++++ 12 files changed, 162 insertions(+), 33 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java index cbe08095128f..880794577033 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; @@ -165,25 +166,29 @@ private ReferenceCountingResourceHolder fetchSegmentInternal( } @Override - public Function, ResourceHolder>> fetchLoadedSegment( + public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( RichSegmentDescriptor segmentDescriptor, String dataSource, ChannelCounters channelCounters ) { - return query -> fetchServedSegmentInternal( + return (query, mappingFunction, closer) -> fetchServedSegmentInternal( segmentDescriptor, dataSource, channelCounters, - query + query, + mappingFunction, + closer ); } - private ResourceHolder> fetchServedSegmentInternal( + private Sequence fetchServedSegmentInternal( RichSegmentDescriptor segmentDescriptor, String dataSource, ChannelCounters channelCounters, - Query query + Query query, + Function, Sequence> mappingFunction, + Closer closer ) { query = query.withDataSource(new TableDataSource(dataSource)); @@ -192,14 +197,23 @@ private ResourceHolder> fetchServedSegmentInternal( ImmutableList.of(segmentDescriptor) ); - DataServerClient dataServerClient = new DataServerClient<>( + DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, new FixedSetServiceLocator(segmentDescriptor.getServers()), objectMapper, smileMapper ); DefaultResponseContext responseContext = new DefaultResponseContext(); - return dataServerClient.run(query, responseContext); + ResourceHolder> clientResponseResourceHolder = dataServerClient.run(query, responseContext); + closer.register(clientResponseResourceHolder); + + Sequence queryReturnSequence = clientResponseResourceHolder.get(); + Sequence parsedResponse = mappingFunction.apply(queryReturnSequence); + + return parsedResponse.map(row -> { + channelCounters.incrementRowCount(); + return row; + }); } private static class SegmentHolder implements Supplier> diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 0ca478cd4dae..e6b8a8d4b47a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -159,7 +159,7 @@ private static Iterator inputSourceSegmentIterator( ); return new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - query -> { + (query, function, closer) -> { throw new AssertionError("Should not be called"); }, segmentId.toDescriptor() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index e6b1489dfaac..0bbba5697472 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -74,7 +74,7 @@ public ReadableInputs attach( segment -> ReadableInput.segment( new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - query -> { + (query, function, closer) -> { throw new AssertionError("Should not be called"); }, DUMMY_SEGMENT_DESCRIPTOR diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index a0e8253c88cf..f5c1521685df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -99,7 +99,7 @@ public ReadableInputs attach( return ResourceHolder.fromCloseable(segment); }, - query -> { + (query, function, closer) -> { throw new AssertionError("Should not be called"); }, SegmentId.dummy(lookupName).toDescriptor() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 3d06199a9805..2f24d6326e01 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -20,8 +20,10 @@ package org.apache.druid.msq.input.table; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; @@ -36,7 +38,7 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; - private final Function, ResourceHolder>> servedSegmentRowsFunction; + private final TriFunction, Function, Sequence>, Closer, Sequence> servedSegmentRowsFunction; private final SegmentDescriptor descriptor; /** @@ -50,7 +52,7 @@ public class SegmentWithDescriptor */ public SegmentWithDescriptor( final Supplier> segmentSupplier, - final Function, ResourceHolder>> servedSegmentRowsFunction, + final TriFunction, Function, Sequence>, Closer, Sequence> servedSegmentRowsFunction, final SegmentDescriptor descriptor ) { @@ -72,10 +74,9 @@ public ResourceHolder getOrLoad() { return segmentSupplier.get(); } - - public ResourceHolder getServedSegmentFromServer(Query query) + public Sequence getServedSegmentFromServer(Query query, Function mappingFunction, Closer closer) { - return servedSegmentRowsFunction.apply(query); + return servedSegmentRowsFunction.apply(query, mappingFunction, closer); } /** * The segment descriptor associated with this physical segment. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java index 5bf9f4f2793f..568c75d73259 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java @@ -19,8 +19,10 @@ package org.apache.druid.msq.querykit; +import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.query.Query; @@ -49,7 +51,7 @@ Supplier> fetchSegment( * Returns a function that fetches the segment corresponding to the provided segmentId from a server where it is * loaded. The segment is not actually fetched until you call {@link Function#apply(Object)}. */ - Function, ResourceHolder>> fetchLoadedSegment( + TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( RichSegmentDescriptor richSegmentDescriptor, String dataSource, ChannelCounters channelCounters diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index ad742c9c21ed..4bb64491ee5a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -55,6 +55,7 @@ import org.apache.druid.timeline.SegmentId; import java.io.IOException; +import java.util.function.Function; /** * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them @@ -112,11 +113,9 @@ private GroupByQuery prepareGroupByQuery(GroupByQuery query) protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { - ResourceHolder servedSegmentFromServer = closer.register( - segment.getServedSegmentFromServer(prepareGroupByQuery(query)) - ); - Sequence rowSequence = servedSegmentFromServer.get(); - resultYielder = Yielders.each(rowSequence); + Sequence servedSegmentFromServer = + segment.getServedSegmentFromServer(prepareGroupByQuery(query), Function.identity(), closer); + resultYielder = Yielders.each(servedSegmentFromServer); } populateFrameWriterAndFlushIfNeeded(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 6880e33de2e1..8f859166ec2e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -162,21 +162,24 @@ public void cleanup() throws IOException closer.close(); } + private static Sequence parseFunction(Object o) { + Sequence inputSequence = (Sequence) o; + return inputSequence.flatMap(resultRow -> { + ScanResultValue scanResultValue = resultRow; + List> events = (List>) scanResultValue.getEvents(); + return Sequences.simple(events); + }).map(List::toArray); + } + @Override protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { - final ResourceHolder resourceHolder = closer.register(segment.getServedSegmentFromServer(query)); - Sequence sequence = resourceHolder.get(); - - Sequence parsedSequence = sequence.flatMap(resultRow -> { - ScanResultValue scanResultValue = resultRow; - List> events = (List>) scanResultValue.getEvents(); - return Sequences.simple(events); - }).map(List::toArray); + final Sequence sequence = segment.getServedSegmentFromServer(query, ScanQueryFrameProcessor::parseFunction, closer); + RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromSequence( - parsedSequence, + sequence, rowSignature ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 1ae16dd94d9d..e68f5a93533d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -25,6 +25,7 @@ import com.google.inject.Injector; import com.google.inject.Module; import com.google.inject.TypeLiteral; +import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -179,7 +180,7 @@ public Supplier> fetchSegment( } @Override - public Function, ResourceHolder>> fetchLoadedSegment( + public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( RichSegmentDescriptor richSegmentDescriptor, String dataSource, ChannelCounters channelCounters diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 3fbe849259cf..cf413957a25b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -36,6 +36,7 @@ import com.google.inject.TypeLiteral; import com.google.inject.util.Modules; import com.google.inject.util.Providers; +import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.config.NullHandling; @@ -432,7 +433,7 @@ public Supplier> fetchSegment( } @Override - public Function, ResourceHolder>> fetchLoadedSegment( + public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( RichSegmentDescriptor segmentDescriptor, String dataSource, ChannelCounters channelCounters diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 164ea97aa635..5859009e0933 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -92,7 +92,7 @@ public ResourceHolder> run(Query query, ResponseContext context) Future resultStreamFuture; try { - resultStreamFuture = Futures.immediateFuture(serviceClient.request( + resultStreamFuture = Futures.immediateFuture(serviceClient.request( //TODO: not immediate? new RequestBuilder(HttpMethod.POST, basePath) .smileContent(smileMapper, query), new DataServerResponseHandler(query, context, smileMapper) diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java new file mode 100644 index 000000000000..dcf722a3001c --- /dev/null +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream; +import org.apache.druid.java.util.http.client.response.ClientResponse; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.query.Query; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.server.QueryResource; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.handler.codec.http.HttpChunk; +import org.jboss.netty.handler.codec.http.HttpResponse; + +import java.io.IOException; +import java.io.InputStream; + +public class DataServerResponseHandler implements HttpResponseHandler +{ + private final String queryId; + private final ResponseContext responseContext; + private final ObjectMapper objectMapper; + + public DataServerResponseHandler(Query query, ResponseContext responseContext, ObjectMapper objectMapper) + { + this.queryId = query.getId(); + this.responseContext = responseContext; + this.objectMapper = objectMapper; + } + + @Override + public ClientResponse handleResponse(HttpResponse response, TrafficCop trafficCop) + { + AppendableByteArrayInputStream in = new AppendableByteArrayInputStream(); + in.add(getContentBytes(response.getContent())); + + try { + final String queryResponseHeaders = response.headers().get(QueryResource.HEADER_RESPONSE_CONTEXT); + if (queryResponseHeaders != null) { + responseContext.merge(ResponseContext.deserialize(queryResponseHeaders, objectMapper)); + } + return ClientResponse.finished(in); + } + catch (IOException e) { + return ClientResponse.finished( + new AppendableByteArrayInputStream() + { + @Override + public int read() throws IOException + { + throw e; + } + } + ); + } + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, + HttpChunk chunk, + long chunkNum + ) + { + clientResponse.getObj().add(getContentBytes(chunk.getContent())); + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + final AppendableByteArrayInputStream obj = clientResponse.getObj(); + obj.done(); + return ClientResponse.finished(obj); + } + + @Override + public void exceptionCaught(ClientResponse clientResponse, Throwable e) + { + final AppendableByteArrayInputStream obj = clientResponse.getObj(); + obj.exceptionCaught(e); + } + + private byte[] getContentBytes(ChannelBuffer content) + { + byte[] contentBytes = new byte[content.readableBytes()]; + content.readBytes(contentBytes); + return contentBytes; + } +} From faeec152e43deb58928dddf56300a07ba2a00a7e Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 25 Sep 2023 09:02:56 +0530 Subject: [PATCH 06/37] Refactor code --- .../msq/exec/LoadedSegmentDataProvider.java | 102 ++++++++++++++++++ .../LoadedSegmentDataProviderFactory.java | 65 +++++++++++ .../msq/exec/TaskDataSegmentProvider.java | 75 +------------ .../org/apache/druid/msq/exec/WorkerImpl.java | 2 +- .../msq/indexing/IndexerFrameContext.java | 11 ++ .../msq/indexing/IndexerWorkerContext.java | 10 +- .../external/ExternalInputSliceReader.java | 4 +- .../input/inline/InlineInputSliceReader.java | 4 +- .../input/lookup/LookupInputSliceReader.java | 4 +- .../input/table/SegmentWithDescriptor.java | 19 ++-- .../input/table/SegmentsInputSliceReader.java | 10 +- .../apache/druid/msq/kernel/FrameContext.java | 2 + .../msq/querykit/DataSegmentProvider.java | 16 --- .../msq/exec/TaskDataSegmentProviderTest.java | 7 +- .../indexing/IndexerWorkerContextTest.java | 1 + .../druid/msq/test/CalciteMSQTestsHelper.java | 29 +---- .../apache/druid/msq/test/MSQTestBase.java | 29 +---- .../druid/msq/test/MSQTestWorkerContext.java | 2 + .../druid/discovery/DataServerClient.java | 2 +- 19 files changed, 219 insertions(+), 175 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java new file mode 100644 index 000000000000..8694e991bd05 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.query.Queries; +import org.apache.druid.query.Query; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.context.DefaultResponseContext; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +public class LoadedSegmentDataProvider +{ + private final RichSegmentDescriptor segmentDescriptor; + private final String dataSource; + private final ChannelCounters channelCounters; + private final ServiceClientFactory serviceClientFactory; + private final CoordinatorClient coordinatorClient; + private final ObjectMapper objectMapper; + private final ObjectMapper smileMapper; + + public LoadedSegmentDataProvider( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters, + ServiceClientFactory serviceClientFactory, + CoordinatorClient coordinatorClient, + ObjectMapper objectMapper, + ObjectMapper smileMapper + ) + { + this.segmentDescriptor = segmentDescriptor; + this.dataSource = dataSource; + this.channelCounters = channelCounters; + this.serviceClientFactory = serviceClientFactory; + this.coordinatorClient = coordinatorClient; + this.objectMapper = objectMapper; + this.smileMapper = smileMapper; + } + + public Sequence fetchServedSegmentInternal( + Query query, + Function, Sequence> mappingFunction, + Closer closer + ) + { + query = query.withDataSource(new TableDataSource(dataSource)); + query = Queries.withSpecificSegments( + query, + ImmutableList.of(segmentDescriptor) + ); + + DataServerClient dataServerClient = new DataServerClient<>( + serviceClientFactory, + new FixedSetServiceLocator(segmentDescriptor.getServers()), + objectMapper, + smileMapper + ); + DefaultResponseContext responseContext = new DefaultResponseContext(); + ResourceHolder> clientResponseResourceHolder = dataServerClient.run(query, responseContext); + closer.register(clientResponseResourceHolder); + + Sequence queryReturnSequence = clientResponseResourceHolder.get(); + + AtomicInteger rowCount = new AtomicInteger(0); + Sequence parsedResponse = mappingFunction.apply(queryReturnSequence).map(row -> { + rowCount.incrementAndGet(); + return row; + }); + closer.register(() -> channelCounters.addFile(rowCount.get(), 0)); + return parsedResponse; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java new file mode 100644 index 000000000000..9e277a37d2b1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.rpc.ServiceClientFactory; + +public class LoadedSegmentDataProviderFactory +{ + private final CoordinatorClient coordinatorClient; + private final ServiceClientFactory serviceClientFactory; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + + public LoadedSegmentDataProviderFactory( + CoordinatorClient coordinatorClient, + ServiceClientFactory serviceClientFactory, + ObjectMapper jsonMapper, + ObjectMapper smileMapper + ) + { + + this.coordinatorClient = coordinatorClient; + this.serviceClientFactory = serviceClientFactory; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + } + + public LoadedSegmentDataProvider createLoadedSegmentDataProvider( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters + ) + { + return new LoadedSegmentDataProvider( + segmentDescriptor, + dataSource, + channelCounters, + serviceClientFactory, + coordinatorClient, + jsonMapper, + smileMapper + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java index 880794577033..c327ec340aea 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -19,28 +19,16 @@ package org.apache.druid.msq.exec; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.DataSegmentProvider; -import org.apache.druid.query.Queries; -import org.apache.druid.query.Query; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.context.DefaultResponseContext; -import org.apache.druid.rpc.FixedSetServiceLocator; -import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -57,7 +45,6 @@ import java.io.IOException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.function.Function; import java.util.function.Supplier; /** @@ -68,27 +55,18 @@ public class TaskDataSegmentProvider implements DataSegmentProvider private final CoordinatorClient coordinatorClient; private final SegmentCacheManager segmentCacheManager; private final IndexIO indexIO; - private final ServiceClientFactory serviceClientFactory; - private final ObjectMapper objectMapper; private final ConcurrentHashMap holders; - private final ObjectMapper smileMapper; public TaskDataSegmentProvider( CoordinatorClient coordinatorClient, SegmentCacheManager segmentCacheManager, - IndexIO indexIO, - ServiceClientFactory serviceClientFactory, - ObjectMapper objectMapper, - ObjectMapper smileMapper + IndexIO indexIO ) { this.coordinatorClient = coordinatorClient; this.segmentCacheManager = segmentCacheManager; this.indexIO = indexIO; - this.serviceClientFactory = serviceClientFactory; - this.objectMapper = objectMapper; this.holders = new ConcurrentHashMap<>(); - this.smileMapper = smileMapper; } @Override @@ -165,57 +143,6 @@ private ReferenceCountingResourceHolder fetchSegmentInternal( } } - @Override - public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( - RichSegmentDescriptor segmentDescriptor, - String dataSource, - ChannelCounters channelCounters - ) - { - return (query, mappingFunction, closer) -> fetchServedSegmentInternal( - segmentDescriptor, - dataSource, - channelCounters, - query, - mappingFunction, - closer - ); - } - - private Sequence fetchServedSegmentInternal( - RichSegmentDescriptor segmentDescriptor, - String dataSource, - ChannelCounters channelCounters, - Query query, - Function, Sequence> mappingFunction, - Closer closer - ) - { - query = query.withDataSource(new TableDataSource(dataSource)); - query = Queries.withSpecificSegments( - query, - ImmutableList.of(segmentDescriptor) - ); - - DataServerClient dataServerClient = new DataServerClient<>( - serviceClientFactory, - new FixedSetServiceLocator(segmentDescriptor.getServers()), - objectMapper, - smileMapper - ); - DefaultResponseContext responseContext = new DefaultResponseContext(); - ResourceHolder> clientResponseResourceHolder = dataServerClient.run(query, responseContext); - closer.register(clientResponseResourceHolder); - - Sequence queryReturnSequence = clientResponseResourceHolder.get(); - Sequence parsedResponse = mappingFunction.apply(queryReturnSequence); - - return parsedResponse.map(row -> { - channelCounters.incrementRowCount(); - return row; - }); - } - private static class SegmentHolder implements Supplier> { private final Supplier> holderSupplier; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 95f656015ac3..4036ad21d1f7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -1102,7 +1102,7 @@ private void makeInputSliceReader() .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) .put(SegmentsInputSlice.class, new SegmentsInputSliceReader( - frameContext.dataSegmentProvider(), + frameContext, MultiStageQueryContext.isReindex(QueryContext.of(task().getContext())) ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index c35832992f93..d522a8a7d88f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.DataSegmentProvider; @@ -38,17 +39,20 @@ public class IndexerFrameContext implements FrameContext private final IndexIO indexIO; private final DataSegmentProvider dataSegmentProvider; private final WorkerMemoryParameters memoryParameters; + private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; public IndexerFrameContext( IndexerWorkerContext context, IndexIO indexIO, DataSegmentProvider dataSegmentProvider, + LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory, WorkerMemoryParameters memoryParameters ) { this.context = context; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; + this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory; this.memoryParameters = memoryParameters; } @@ -76,6 +80,13 @@ public DataSegmentProvider dataSegmentProvider() return dataSegmentProvider; } + @Override + public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + { + return loadedSegmentDataProviderFactory; + } + + @Override public File tempDir() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 386a1f99af88..57b843146ca0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.exec.TaskDataSegmentProvider; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; @@ -69,6 +70,7 @@ public class IndexerWorkerContext implements WorkerContext private final Injector injector; private final IndexIO indexIO; private final TaskDataSegmentProvider dataSegmentProvider; + private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; private final ServiceClientFactory clientFactory; @GuardedBy("this") @@ -82,6 +84,7 @@ public IndexerWorkerContext( final Injector injector, final IndexIO indexIO, final TaskDataSegmentProvider dataSegmentProvider, + final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory, final ServiceClientFactory clientFactory ) { @@ -89,6 +92,7 @@ public IndexerWorkerContext( this.injector = injector; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; + this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory; this.clientFactory = clientFactory; } @@ -109,7 +113,10 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to new TaskDataSegmentProvider( toolbox.getCoordinatorClient(), segmentCacheManager, - indexIO, + indexIO + ), + new LoadedSegmentDataProviderFactory( + toolbox.getCoordinatorClient(), serviceClientFactory, toolbox.getJsonMapper(), smileMapper @@ -236,6 +243,7 @@ public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) this, indexIO, dataSegmentProvider, + loadedSegmentDataProviderFactory, WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index e6b8a8d4b47a..7f4cd6ffb895 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -159,9 +159,7 @@ private static Iterator inputSourceSegmentIterator( ); return new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - (query, function, closer) -> { - throw new AssertionError("Should not be called"); - }, + null, segmentId.toDescriptor() ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 0bbba5697472..40d4a2237382 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -74,9 +74,7 @@ public ReadableInputs attach( segment -> ReadableInput.segment( new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - (query, function, closer) -> { - throw new AssertionError("Should not be called"); - }, + null, DUMMY_SEGMENT_DESCRIPTOR ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index f5c1521685df..ecf0c0d70cf0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -99,9 +99,7 @@ public ReadableInputs attach( return ResourceHolder.fromCloseable(segment); }, - (query, function, closer) -> { - throw new AssertionError("Should not be called"); - }, + null, SegmentId.dummy(lookupName).toDescriptor() ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 2f24d6326e01..2f77589da488 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -20,10 +20,10 @@ package org.apache.druid.msq.input.table; import com.google.common.base.Preconditions; -import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; @@ -38,7 +38,7 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; - private final TriFunction, Function, Sequence>, Closer, Sequence> servedSegmentRowsFunction; + private final LoadedSegmentDataProvider loadedSegmentDataProvider; private final SegmentDescriptor descriptor; /** @@ -46,18 +46,17 @@ public class SegmentWithDescriptor * * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} * logic must include a delegated call to {@link Segment#close()}. - * @param servedSegmentRowsFunction function which accepts a {@link Query} and fetches a {@link Sequence} of results - * from a server where the segment is served. + * @param loadedSegmentDataProvider * @param descriptor segment descriptor */ public SegmentWithDescriptor( final Supplier> segmentSupplier, - final TriFunction, Function, Sequence>, Closer, Sequence> servedSegmentRowsFunction, + final LoadedSegmentDataProvider loadedSegmentDataProvider, final SegmentDescriptor descriptor ) { this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment"); - this.servedSegmentRowsFunction = servedSegmentRowsFunction; + this.loadedSegmentDataProvider = loadedSegmentDataProvider; this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor"); } @@ -74,9 +73,13 @@ public ResourceHolder getOrLoad() { return segmentSupplier.get(); } - public Sequence getServedSegmentFromServer(Query query, Function mappingFunction, Closer closer) + public Sequence getServedSegmentFromServer( + Query query, + Function, Sequence> mappingFunction, + Closer closer + ) { - return servedSegmentRowsFunction.apply(query, mappingFunction, closer); + return loadedSegmentDataProvider.fetchServedSegmentInternal(query, mappingFunction, closer); } /** * The segment descriptor associated with this physical segment. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index d310d4af3d37..c0795de3311c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -23,10 +23,12 @@ import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.timeline.SegmentId; @@ -40,11 +42,13 @@ public class SegmentsInputSliceReader implements InputSliceReader { private final DataSegmentProvider dataSegmentProvider; + private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; private final boolean isReindex; - public SegmentsInputSliceReader(final DataSegmentProvider dataSegmentProvider, final boolean isReindex) + public SegmentsInputSliceReader(final FrameContext frameContext, final boolean isReindex) { - this.dataSegmentProvider = dataSegmentProvider; + this.dataSegmentProvider = frameContext.dataSegmentProvider(); + this.loadedSegmentDataProviderFactory = frameContext.loadedSegmentDataProviderFactory(); this.isReindex = isReindex; } @@ -94,7 +98,7 @@ private Iterator dataSegmentIterator( return new SegmentWithDescriptor( dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), - dataSegmentProvider.fetchLoadedSegment(descriptor, dataSource, channelCounters), + loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(descriptor, dataSource, channelCounters), descriptor ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index 2339ac5537a0..49871cecc1d4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.kernel; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; @@ -43,6 +44,7 @@ public interface FrameContext RowIngestionMeters rowIngestionMeters(); DataSegmentProvider dataSegmentProvider(); + LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory(); File tempDir(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java index 568c75d73259..91ee4a487885 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java @@ -19,17 +19,11 @@ package org.apache.druid.msq.querykit; -import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.query.Query; import org.apache.druid.segment.Segment; import org.apache.druid.timeline.SegmentId; -import java.util.function.Function; import java.util.function.Supplier; public interface DataSegmentProvider @@ -46,14 +40,4 @@ Supplier> fetchSegment( ChannelCounters channelCounters, boolean isReindex ); - - /** - * Returns a function that fetches the segment corresponding to the provided segmentId from a server where it is - * loaded. The segment is not actually fetched until you call {@link Function#apply(Object)}. - */ - TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( - RichSegmentDescriptor richSegmentDescriptor, - String dataSource, - ChannelCounters channelCounters - ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index 7c1fa40cce39..eafb443bafc1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -42,7 +42,6 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -70,7 +69,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.File; @@ -152,10 +150,7 @@ public void setUp() throws Exception provider = new TaskDataSegmentProvider( new TestCoordinatorClientImpl(), cacheManager, - indexIO, - Mockito.mock(ServiceClientFactory.class), - jsonMapper, - jsonMapper + indexIO ); exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(THREADS, getClass().getSimpleName() + "-%s")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java index 0ea9ab45f482..2ae8d155d4dc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java @@ -49,6 +49,7 @@ public void setup() injectorMock, null, null, + null, null ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index e68f5a93533d..0c840b14a88e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -25,7 +25,6 @@ import com.google.inject.Injector; import com.google.inject.Module; import com.google.inject.TypeLiteral; -import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -39,17 +38,13 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.ForwardingQueryProcessingPool; -import org.apache.druid.query.Query; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -89,7 +84,6 @@ import java.io.IOException; import java.util.List; import java.util.Set; -import java.util.function.Function; import java.util.function.Supplier; import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE1; @@ -167,28 +161,7 @@ public String getFormatString() )); binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bind(DataSegmentProvider.class) - .toInstance(new DataSegmentProvider() - { - @Override - public Supplier> fetchSegment( - SegmentId segmentId, - ChannelCounters channelCounters, - boolean isReindex - ) - { - return getSupplierForSegment(segmentId); - } - - @Override - public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( - RichSegmentDescriptor richSegmentDescriptor, - String dataSource, - ChannelCounters channelCounters - ) - { - throw new AssertionError(); - } - }); + .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index cf413957a25b..b9037a949c45 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -36,7 +36,6 @@ import com.google.inject.TypeLiteral; import com.google.inject.util.Modules; import com.google.inject.util.Providers; -import org.apache.commons.lang3.function.TriFunction; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.config.NullHandling; @@ -72,14 +71,12 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.input.InputSourceModule; -import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; @@ -107,7 +104,6 @@ import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; @@ -117,7 +113,6 @@ import org.apache.druid.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.ForwardingQueryProcessingPool; -import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -210,7 +205,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -420,28 +414,7 @@ public String getFormatString() binder.bind(QueryProcessingPool.class) .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(DataSegmentProvider.class) - .toInstance(new DataSegmentProvider() - { - @Override - public Supplier> fetchSegment( - SegmentId segmentId, - ChannelCounters channelCounters, - boolean isReindex - ) - { - return getSupplierForSegment(segmentId); - } - - @Override - public TriFunction, Function, Sequence>, Closer, Sequence> fetchLoadedSegment( - RichSegmentDescriptor segmentDescriptor, - String dataSource, - ChannelCounters channelCounters - ) - { - throw new AssertionError(); - } - }); + .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index a6f98b3ba85f..26a81ec81758 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -153,10 +153,12 @@ public void setObjectMapper(ObjectMapper objectMapper) injector, indexIO, null, + null, null ), indexIO, injector.getInstance(DataSegmentProvider.class), + null, workerMemoryParameters ); } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 5859009e0933..164ea97aa635 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -92,7 +92,7 @@ public ResourceHolder> run(Query query, ResponseContext context) Future resultStreamFuture; try { - resultStreamFuture = Futures.immediateFuture(serviceClient.request( //TODO: not immediate? + resultStreamFuture = Futures.immediateFuture(serviceClient.request( new RequestBuilder(HttpMethod.POST, basePath) .smileContent(smileMapper, query), new DataServerResponseHandler(query, context, smileMapper) From 0afb4d361b22cc77eafc004eac7f85bbf50270cc Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 26 Sep 2023 09:25:50 +0530 Subject: [PATCH 07/37] Cover handoff cases --- .../druid/msq/exec/HandoffException.java | 24 +++ .../msq/exec/LoadedSegmentDataProvider.java | 77 +-------- .../LoadedSegmentDataProviderFactory.java | 2 +- .../exec/LoadedSegmentDataProviderImpl.java | 153 ++++++++++++++++++ .../input/table/SegmentWithDescriptor.java | 7 +- .../GroupByPreShuffleFrameProcessor.java | 2 +- .../scan/ScanQueryFrameProcessor.java | 13 +- .../druid/discovery/DataServerClient.java | 82 ++++------ .../druid/rpc/FixedSetServiceLocator.java | 2 +- 9 files changed, 229 insertions(+), 133 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java new file mode 100644 index 000000000000..c656e7696339 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +public class HandoffException extends Exception +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 8694e991bd05..67fc642d38d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -19,84 +19,17 @@ package org.apache.druid.msq.exec; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.query.Queries; import org.apache.druid.query.Query; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.context.DefaultResponseContext; -import org.apache.druid.rpc.FixedSetServiceLocator; -import org.apache.druid.rpc.ServiceClientFactory; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; -public class LoadedSegmentDataProvider +public interface LoadedSegmentDataProvider { - private final RichSegmentDescriptor segmentDescriptor; - private final String dataSource; - private final ChannelCounters channelCounters; - private final ServiceClientFactory serviceClientFactory; - private final CoordinatorClient coordinatorClient; - private final ObjectMapper objectMapper; - private final ObjectMapper smileMapper; - - public LoadedSegmentDataProvider( - RichSegmentDescriptor segmentDescriptor, - String dataSource, - ChannelCounters channelCounters, - ServiceClientFactory serviceClientFactory, - CoordinatorClient coordinatorClient, - ObjectMapper objectMapper, - ObjectMapper smileMapper - ) - { - this.segmentDescriptor = segmentDescriptor; - this.dataSource = dataSource; - this.channelCounters = channelCounters; - this.serviceClientFactory = serviceClientFactory; - this.coordinatorClient = coordinatorClient; - this.objectMapper = objectMapper; - this.smileMapper = smileMapper; - } - - public Sequence fetchServedSegmentInternal( - Query query, - Function, Sequence> mappingFunction, + Sequence fetchRowsFromDataServer( + Query query, + Function, Sequence> mappingFunction, Closer closer - ) - { - query = query.withDataSource(new TableDataSource(dataSource)); - query = Queries.withSpecificSegments( - query, - ImmutableList.of(segmentDescriptor) - ); - - DataServerClient dataServerClient = new DataServerClient<>( - serviceClientFactory, - new FixedSetServiceLocator(segmentDescriptor.getServers()), - objectMapper, - smileMapper - ); - DefaultResponseContext responseContext = new DefaultResponseContext(); - ResourceHolder> clientResponseResourceHolder = dataServerClient.run(query, responseContext); - closer.register(clientResponseResourceHolder); - - Sequence queryReturnSequence = clientResponseResourceHolder.get(); - - AtomicInteger rowCount = new AtomicInteger(0); - Sequence parsedResponse = mappingFunction.apply(queryReturnSequence).map(row -> { - rowCount.incrementAndGet(); - return row; - }); - closer.register(() -> channelCounters.addFile(rowCount.get(), 0)); - return parsedResponse; - } + ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 9e277a37d2b1..62a4e407097d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -52,7 +52,7 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( ChannelCounters channelCounters ) { - return new LoadedSegmentDataProvider( + return new LoadedSegmentDataProviderImpl( segmentDescriptor, dataSource, channelCounters, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java new file mode 100644 index 000000000000..43342cac6e13 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.query.Queries; +import org.apache.druid.query.Query; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.context.DefaultResponseContext; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; + +import java.util.List; +import java.util.function.Function; + +public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider +{ + private final static int DEFAULT_NUM_TRIES = 1; + private final RichSegmentDescriptor segmentDescriptor; + private final String dataSource; + private final ChannelCounters channelCounters; + private final ServiceClientFactory serviceClientFactory; + private final CoordinatorClient coordinatorClient; + private final ObjectMapper objectMapper; + private final ObjectMapper smileMapper; + + public LoadedSegmentDataProviderImpl( + RichSegmentDescriptor segmentDescriptor, + String dataSource, + ChannelCounters channelCounters, + ServiceClientFactory serviceClientFactory, + CoordinatorClient coordinatorClient, + ObjectMapper objectMapper, + ObjectMapper smileMapper + ) + { + this.segmentDescriptor = segmentDescriptor; + this.dataSource = dataSource; + this.channelCounters = channelCounters; + this.serviceClientFactory = serviceClientFactory; + this.coordinatorClient = coordinatorClient; + this.objectMapper = objectMapper; + this.smileMapper = smileMapper; + } + + @Override + public Sequence fetchRowsFromDataServer( + Query query, + Function, Sequence> mappingFunction, + Closer closer + ) + { + final Query preparedQuery = Queries.withSpecificSegments( + query.withDataSource(new TableDataSource(dataSource)), + ImmutableList.of(segmentDescriptor) + ); + + final DataServerClient dataServerClient = new DataServerClient<>( + serviceClientFactory, + new FixedSetServiceLocator(segmentDescriptor.getServers()), + objectMapper, + smileMapper + ); + + final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); + final ResponseContext responseContext = new DefaultResponseContext(); + + Sequence queryReturnSequence; + try { + queryReturnSequence = RetryUtils.retry( + () -> { + Sequence sequence = dataServerClient.run(preparedQuery, responseContext, closer); + final List missingSegments = getMissingSegments(responseContext); + if (missingSegments.isEmpty()) { + return sequence; + } else { + Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); + if (Boolean.TRUE.equals(wasHandedOff)) { + throw new HandoffException(); + } else { + throw new ISE( + "Segment[%s] could not be found on data server, but segment was not handed off.", + segmentDescriptor + ); + } + } + }, + input -> !(input instanceof HandoffException), + numRetriesOnMissingSegments + ); + } catch (Exception e) { + throw new RE(e); + } + + return mappingFunction.apply(queryReturnSequence).map(row -> { + channelCounters.incrementRowCount(); + return row; + }); + } + + private static List getMissingSegments(final ResponseContext responseContext) + { + List missingSegments = responseContext.getMissingSegments(); + if (missingSegments == null) { + return ImmutableList.of(); + } + return missingSegments; + } + + private static boolean checkSegmentHandoff( + CoordinatorClient coordinatorClient, + String dataSource, + SegmentDescriptor segmentDescriptor + ) throws Exception { + Boolean wasHandedOff = RetryUtils.retry( + () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), + input -> true, + RetryUtils.DEFAULT_MAX_TRIES + ); + + return Boolean.TRUE.equals(wasHandedOff); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 2f77589da488..ccc878fe993e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -73,14 +73,15 @@ public ResourceHolder getOrLoad() { return segmentSupplier.get(); } - public Sequence getServedSegmentFromServer( + public Sequence fetchRowsFromDataServer( Query query, - Function, Sequence> mappingFunction, + Function, Sequence> mappingFunction, Closer closer ) { - return loadedSegmentDataProvider.fetchServedSegmentInternal(query, mappingFunction, closer); + return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, closer); } + /** * The segment descriptor associated with this physical segment. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 4bb64491ee5a..131715dfd911 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -114,7 +114,7 @@ protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment { if (resultYielder == null) { Sequence servedSegmentFromServer = - segment.getServedSegmentFromServer(prepareGroupByQuery(query), Function.identity(), closer); + segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), closer); resultYielder = Yielders.each(servedSegmentFromServer); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 8f859166ec2e..c9a512e256f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -162,11 +162,10 @@ public void cleanup() throws IOException closer.close(); } - private static Sequence parseFunction(Object o) { - Sequence inputSequence = (Sequence) o; + private static Sequence mappingFunction(Sequence inputSequence) + { return inputSequence.flatMap(resultRow -> { - ScanResultValue scanResultValue = resultRow; - List> events = (List>) scanResultValue.getEvents(); + List> events = (List>) resultRow.getEvents(); return Sequences.simple(events); }).map(List::toArray); } @@ -175,7 +174,11 @@ private static Sequence parseFunction(Object o) { protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { - final Sequence sequence = segment.getServedSegmentFromServer(query, ScanQueryFrameProcessor::parseFunction, closer); + final Sequence sequence = segment.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + closer + ); RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromSequence( diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 164ea97aa635..ddc51a3d5447 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -21,12 +21,11 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.Futures; import org.apache.druid.client.JsonParserIterator; -import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; @@ -78,8 +77,7 @@ public DataServerClient( this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } - - public ResourceHolder> run(Query query, ResponseContext context) + public Sequence run(Query query, ResponseContext responseContext, Closer closer) { final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); @@ -91,58 +89,42 @@ public ResourceHolder> run(Query query, ResponseContext context) } Future resultStreamFuture; - try { - resultStreamFuture = Futures.immediateFuture(serviceClient.request( - new RequestBuilder(HttpMethod.POST, basePath) - .smileContent(smileMapper, query), - new DataServerResponseHandler(query, context, smileMapper) - )); - - BaseSequence> baseSequence = new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public JsonParserIterator make() - { - return new JsonParserIterator<>( - queryResultType, - resultStreamFuture, - basePath, - query, - "", - smileMapper - ); - } - - @Override - public void cleanup(JsonParserIterator iterFromMake) - { - CloseableUtils.closeAndWrapExceptions(iterFromMake); - } + resultStreamFuture = serviceClient.asyncRequest( + new RequestBuilder(HttpMethod.POST, basePath) + .smileContent(smileMapper, query), + new DataServerResponseHandler(query, responseContext, smileMapper) + ); + closer.register( + () -> { + if (!resultStreamFuture.isDone()) { + cancelQuery(query, cancelPath); } - ); - - return new ResourceHolder>() - { - @Override - public Sequence get() - { - return baseSequence; } + ); - @Override - public void close() + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() { - if (!resultStreamFuture.isDone()) { - cancelQuery(query, cancelPath); + @Override + public JsonParserIterator make() + { + return new JsonParserIterator<>( + queryResultType, + resultStreamFuture, + basePath, + query, + "", + smileMapper + ); + } + + @Override + public void cleanup(JsonParserIterator iterFromMake) + { + CloseableUtils.closeAndWrapExceptions(iterFromMake); } } - }; - } - catch (Exception e) { - cancelQuery(query, cancelPath); - throw new RuntimeException(e); - } + ); } private void cancelQuery(Query query, String cancelPath) diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java index fb6d0075c39d..33d4cdd89d56 100644 --- a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -58,7 +58,7 @@ public ListenableFuture locate() return Futures.immediateFuture( ServiceLocations.forLocation( locationSet.stream() - .skip(new ThreadLocalRandom().nextInt(locationSet.size())) + .skip(ThreadLocalRandom.current().nextInt(locationSet.size())) .findFirst() .orElse(null) ) From 46f84c8846e82c43a5ff6f48e3d743635e50b316 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 26 Sep 2023 10:41:55 +0530 Subject: [PATCH 08/37] Resolve bug --- .../org/apache/druid/msq/exec/HandoffException.java | 2 +- .../druid/msq/exec/LoadedSegmentDataProvider.java | 3 ++- .../msq/exec/LoadedSegmentDataProviderImpl.java | 12 ++++++++---- .../druid/msq/input/table/SegmentWithDescriptor.java | 3 ++- .../druid/msq/querykit/BaseLeafFrameProcessor.java | 7 ++++++- .../org/apache/druid/discovery/DataServerClient.java | 9 +++++---- 6 files changed, 24 insertions(+), 12 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java index c656e7696339..8a0bdd59ba56 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java @@ -19,6 +19,6 @@ package org.apache.druid.msq.exec; -public class HandoffException extends Exception +public class HandoffException extends RuntimeException { } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 67fc642d38d6..be196ba10dee 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; +import java.io.IOException; import java.util.function.Function; public interface LoadedSegmentDataProvider @@ -31,5 +32,5 @@ Sequence fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, Closer closer - ); + ) throws IOException; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 43342cac6e13..d6686ea63750 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -20,12 +20,13 @@ package org.apache.druid.msq.exec; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; @@ -40,6 +41,7 @@ import org.apache.druid.rpc.FixedSetServiceLocator; import org.apache.druid.rpc.ServiceClientFactory; +import java.io.IOException; import java.util.List; import java.util.function.Function; @@ -78,7 +80,7 @@ public Sequence fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, Closer closer - ) + ) throws IOException { final Query preparedQuery = Queries.withSpecificSegments( query.withDataSource(new TableDataSource(dataSource)), @@ -118,8 +120,10 @@ public Sequence fetchRowsFromDataServer( input -> !(input instanceof HandoffException), numRetriesOnMissingSegments ); - } catch (Exception e) { - throw new RE(e); + } + catch (Exception e) { + Throwables.propagateIfPossible(e, HandoffException.class); + throw new IOE(e, "Exception while fetching rows from dataservers."); } return mappingFunction.apply(queryReturnSequence).map(row -> { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index ccc878fe993e..5a467e16b3d5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -28,6 +28,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; +import java.io.IOException; import java.util.Objects; import java.util.function.Function; import java.util.function.Supplier; @@ -77,7 +78,7 @@ public Sequence fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, Closer closer - ) + ) throws IOException { return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, closer); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 1ea87f89db0b..0c9f1f149d13 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -33,6 +33,7 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.msq.exec.HandoffException; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.DataSource; @@ -158,7 +159,11 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws } else if (baseInput.hasSegment()) { SegmentWithDescriptor segment = baseInput.getSegment(); if (segment.isRealtimeSegment()) { - return runWithLoadedSegment(baseInput.getSegment()); + try { + return runWithLoadedSegment(baseInput.getSegment()); + } catch (HandoffException handoffException) { + return runWithSegment(baseInput.getSegment()); + } } else { return runWithSegment(baseInput.getSegment()); } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index ddc51a3d5447..3199dd4561b0 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -78,6 +79,7 @@ public DataServerClient( } public Sequence run(Query query, ResponseContext responseContext, Closer closer) + throws Exception { final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); @@ -88,15 +90,14 @@ public Sequence run(Query query, ResponseContext responseContext, Closer c queryResultType = objectMapper.getTypeFactory().constructType(ResultRow.class); } - Future resultStreamFuture; - resultStreamFuture = serviceClient.asyncRequest( + InputStream resultStream = serviceClient.request( new RequestBuilder(HttpMethod.POST, basePath) .smileContent(smileMapper, query), new DataServerResponseHandler(query, responseContext, smileMapper) ); closer.register( () -> { - if (!resultStreamFuture.isDone()) { + if (!resultStream.markSupported()) { cancelQuery(query, cancelPath); } } @@ -110,7 +111,7 @@ public JsonParserIterator make() { return new JsonParserIterator<>( queryResultType, - resultStreamFuture, + Futures.immediateFuture(resultStream), basePath, query, "", From 3183f3715c969cfa8df2147c8c6d055a347b171d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 27 Sep 2023 17:25:56 +0530 Subject: [PATCH 09/37] Change handoff from exception to return --- .../druid/msq/exec/HandoffException.java | 24 ---------- .../msq/exec/LoadedSegmentDataProvider.java | 13 +++-- .../exec/LoadedSegmentDataProviderImpl.java | 43 +++++++++-------- .../input/table/SegmentWithDescriptor.java | 16 ++++--- .../msq/querykit/BaseLeafFrameProcessor.java | 7 +-- .../GroupByPreShuffleFrameProcessor.java | 11 +++-- .../scan/ScanQueryFrameProcessor.java | 20 +++++--- .../druid/discovery/DataServerClient.java | 48 ++++++++++--------- 8 files changed, 90 insertions(+), 92 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java deleted file mode 100644 index 8a0bdd59ba56..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/HandoffException.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.msq.exec; - -public class HandoffException extends RuntimeException -{ -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index be196ba10dee..98282d5a6c9f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -19,8 +19,8 @@ package org.apache.druid.msq.exec; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; import java.io.IOException; @@ -28,9 +28,16 @@ public interface LoadedSegmentDataProvider { - Sequence fetchRowsFromDataServer( + Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Closer closer + Class queryResultType ) throws IOException; + + enum DataServerQueryStatus + { + SUCCESS, + HANDOFF, + FAILED + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index d6686ea63750..10a666d94347 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -19,17 +19,17 @@ package org.apache.druid.msq.exec; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.query.Queries; @@ -47,7 +47,7 @@ public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider { - private final static int DEFAULT_NUM_TRIES = 1; + private static final int DEFAULT_NUM_TRIES = 5; private final RichSegmentDescriptor segmentDescriptor; private final String dataSource; private final ChannelCounters channelCounters; @@ -76,10 +76,10 @@ public LoadedSegmentDataProviderImpl( } @Override - public Sequence fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Closer closer + Class resultClass ) throws IOException { final Query preparedQuery = Queries.withSpecificSegments( @@ -90,25 +90,26 @@ public Sequence fetchRowsFromDataServer( final DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, new FixedSetServiceLocator(segmentDescriptor.getServers()), - objectMapper, smileMapper ); + final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); final ResponseContext responseContext = new DefaultResponseContext(); - Sequence queryReturnSequence; + Pair> statusSequencePair; try { - queryReturnSequence = RetryUtils.retry( + statusSequencePair = RetryUtils.retry( () -> { - Sequence sequence = dataServerClient.run(preparedQuery, responseContext, closer); + Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); final List missingSegments = getMissingSegments(responseContext); if (missingSegments.isEmpty()) { - return sequence; + // Segment was found + return Pair.of(DataServerQueryStatus.SUCCESS, sequence); } else { Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); if (Boolean.TRUE.equals(wasHandedOff)) { - throw new HandoffException(); + return Pair.of(DataServerQueryStatus.HANDOFF, null); } else { throw new ISE( "Segment[%s] could not be found on data server, but segment was not handed off.", @@ -117,19 +118,22 @@ public Sequence fetchRowsFromDataServer( } } }, - input -> !(input instanceof HandoffException), + input -> true, numRetriesOnMissingSegments ); + + return Pair.of( + statusSequencePair.lhs, + mappingFunction.apply(statusSequencePair.rhs) + .map(row -> { + channelCounters.incrementRowCount(); + return row; + }) + ); } catch (Exception e) { - Throwables.propagateIfPossible(e, HandoffException.class); throw new IOE(e, "Exception while fetching rows from dataservers."); } - - return mappingFunction.apply(queryReturnSequence).map(row -> { - channelCounters.incrementRowCount(); - return row; - }); } private static List getMissingSegments(final ResponseContext responseContext) @@ -145,7 +149,8 @@ private static boolean checkSegmentHandoff( CoordinatorClient coordinatorClient, String dataSource, SegmentDescriptor segmentDescriptor - ) throws Exception { + ) throws Exception + { Boolean wasHandedOff = RetryUtils.retry( () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), input -> true, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 5a467e16b3d5..4ccd140bb945 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -21,8 +21,8 @@ import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; @@ -47,7 +47,8 @@ public class SegmentWithDescriptor * * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} * logic must include a delegated call to {@link Segment#close()}. - * @param loadedSegmentDataProvider + * @param loadedSegmentDataProvider {@link LoadedSegmentDataProvider} which fetches the corresponding results from a + * data server where the segment is loaded. The call will fetch the * @param descriptor segment descriptor */ public SegmentWithDescriptor( @@ -74,13 +75,14 @@ public ResourceHolder getOrLoad() { return segmentSupplier.get(); } - public Sequence fetchRowsFromDataServer( - Query query, - Function, Sequence> mappingFunction, - Closer closer + + public Pair> fetchRowsFromDataServer( + Query query, + Function, Sequence> mappingFunction, + Class queryResultType ) throws IOException { - return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, closer); + return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, queryResultType); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 0c9f1f149d13..1ea87f89db0b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -33,7 +33,6 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.msq.exec.HandoffException; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.DataSource; @@ -159,11 +158,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws } else if (baseInput.hasSegment()) { SegmentWithDescriptor segment = baseInput.getSegment(); if (segment.isRealtimeSegment()) { - try { - return runWithLoadedSegment(baseInput.getSegment()); - } catch (HandoffException handoffException) { - return runWithSegment(baseInput.getSegment()); - } + return runWithLoadedSegment(baseInput.getSegment()); } else { return runWithSegment(baseInput.getSegment()); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 131715dfd911..a3ed5bab09d2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -35,10 +35,12 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; @@ -113,9 +115,12 @@ private GroupByQuery prepareGroupByQuery(GroupByQuery query) protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { - Sequence servedSegmentFromServer = - segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), closer); - resultYielder = Yielders.each(servedSegmentFromServer); + Pair> statusSequencePair = + segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class); + if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { + return runWithSegment(segment); + } + resultYielder = Yielders.each(statusSequencePair.rhs); } populateFrameWriterAndFlushIfNeeded(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index c9a512e256f8..1ff4467a20bc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -42,12 +42,14 @@ import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.msq.input.ParseExceptionUtils; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.external.ExternalSegment; @@ -162,7 +164,7 @@ public void cleanup() throws IOException closer.close(); } - private static Sequence mappingFunction(Sequence inputSequence) + public static Sequence mappingFunction(Sequence inputSequence) { return inputSequence.flatMap(resultRow -> { List> events = (List>) resultRow.getEvents(); @@ -174,15 +176,19 @@ private static Sequence mappingFunction(Sequence inpu protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { - final Sequence sequence = segment.fetchRowsFromDataServer( - query, - ScanQueryFrameProcessor::mappingFunction, - closer - ); + final Pair> statusSequencePair = + segment.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + ScanResultValue.class + ); + if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { + return runWithSegment(segment); + } RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromSequence( - sequence, + statusSequencePair.rhs, rowSignature ); diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 3199dd4561b0..da3a36ad1337 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -21,20 +21,18 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.query.Query; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.rpc.FixedSetServiceLocator; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; @@ -57,50 +55,54 @@ public class DataServerClient private static final Logger log = new Logger(DataServerClient.class); private static final String SERVED_SEGMENT_CLIENT_NAME = "ServedSegmentClient"; private final ServiceClient serviceClient; - private final ObjectMapper objectMapper; private final ObjectMapper smileMapper; private final ScheduledExecutorService queryCancellationExecutor; public DataServerClient( ServiceClientFactory serviceClientFactory, FixedSetServiceLocator fixedSetServiceLocator, - ObjectMapper objectMapper, ObjectMapper smileMapper ) { serviceClient = serviceClientFactory.makeClient( SERVED_SEGMENT_CLIENT_NAME, fixedSetServiceLocator, - StandardRetryPolicy.aboutAnHour() + StandardRetryPolicy.noRetries() ); - this.objectMapper = objectMapper; this.smileMapper = smileMapper; this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } - public Sequence run(Query query, ResponseContext responseContext, Closer closer) - throws Exception + public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType) { final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); - JavaType queryResultType; - if (query instanceof ScanQuery) { - queryResultType = objectMapper.getTypeFactory().constructType(ScanResultValue.class); - } else { - queryResultType = objectMapper.getTypeFactory().constructType(ResultRow.class); - } - InputStream resultStream = serviceClient.request( + ListenableFuture resultStreamFuture = serviceClient.asyncRequest( new RequestBuilder(HttpMethod.POST, basePath) .smileContent(smileMapper, query), new DataServerResponseHandler(query, responseContext, smileMapper) ); - closer.register( - () -> { - if (!resultStream.markSupported()) { - cancelQuery(query, cancelPath); + + Futures.addCallback( + resultStreamFuture, + new FutureCallback() + { + @Override + public void onSuccess(InputStream result) + { + // Do nothing } - } + + @Override + public void onFailure(Throwable t) + { + if (resultStreamFuture.isCancelled()) { + cancelQuery(query, cancelPath); + } + } + }, + Execs.directExecutor() ); return new BaseSequence<>( @@ -111,7 +113,7 @@ public JsonParserIterator make() { return new JsonParserIterator<>( queryResultType, - Futures.immediateFuture(resultStream), + resultStreamFuture, basePath, query, "", From 2570840c79d8fdcd93abed00df239d35b13ca408 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 27 Sep 2023 17:36:17 +0530 Subject: [PATCH 10/37] Add smile check --- .../LoadedSegmentDataProviderFactory.java | 12 ++++------ .../exec/LoadedSegmentDataProviderImpl.java | 7 ++---- .../msq/indexing/IndexerWorkerContext.java | 1 - .../druid/discovery/DataServerClient.java | 22 +++++++++++++------ 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 62a4e407097d..7bfd4c8d1612 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -29,21 +29,18 @@ public class LoadedSegmentDataProviderFactory { private final CoordinatorClient coordinatorClient; private final ServiceClientFactory serviceClientFactory; - private final ObjectMapper jsonMapper; - private final ObjectMapper smileMapper; + private final ObjectMapper objectMapper; public LoadedSegmentDataProviderFactory( CoordinatorClient coordinatorClient, ServiceClientFactory serviceClientFactory, - ObjectMapper jsonMapper, - ObjectMapper smileMapper + ObjectMapper objectMapper ) { this.coordinatorClient = coordinatorClient; this.serviceClientFactory = serviceClientFactory; - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; + this.objectMapper = objectMapper; } public LoadedSegmentDataProvider createLoadedSegmentDataProvider( @@ -58,8 +55,7 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( channelCounters, serviceClientFactory, coordinatorClient, - jsonMapper, - smileMapper + objectMapper ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 10a666d94347..5e8fdfc2f9f7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -54,7 +54,6 @@ public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider private final ServiceClientFactory serviceClientFactory; private final CoordinatorClient coordinatorClient; private final ObjectMapper objectMapper; - private final ObjectMapper smileMapper; public LoadedSegmentDataProviderImpl( RichSegmentDescriptor segmentDescriptor, @@ -62,8 +61,7 @@ public LoadedSegmentDataProviderImpl( ChannelCounters channelCounters, ServiceClientFactory serviceClientFactory, CoordinatorClient coordinatorClient, - ObjectMapper objectMapper, - ObjectMapper smileMapper + ObjectMapper objectMapper ) { this.segmentDescriptor = segmentDescriptor; @@ -72,7 +70,6 @@ public LoadedSegmentDataProviderImpl( this.serviceClientFactory = serviceClientFactory; this.coordinatorClient = coordinatorClient; this.objectMapper = objectMapper; - this.smileMapper = smileMapper; } @Override @@ -90,7 +87,7 @@ public Pair> final DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, new FixedSetServiceLocator(segmentDescriptor.getServers()), - smileMapper + objectMapper ); final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 57b843146ca0..b237babf6df6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -118,7 +118,6 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to new LoadedSegmentDataProviderFactory( toolbox.getCoordinatorClient(), serviceClientFactory, - toolbox.getJsonMapper(), smileMapper ), serviceClientFactory diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index da3a36ad1337..8bed17ef9bb1 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -55,13 +56,13 @@ public class DataServerClient private static final Logger log = new Logger(DataServerClient.class); private static final String SERVED_SEGMENT_CLIENT_NAME = "ServedSegmentClient"; private final ServiceClient serviceClient; - private final ObjectMapper smileMapper; + private final ObjectMapper objectMapper; private final ScheduledExecutorService queryCancellationExecutor; public DataServerClient( ServiceClientFactory serviceClientFactory, FixedSetServiceLocator fixedSetServiceLocator, - ObjectMapper smileMapper + ObjectMapper objectMapper ) { serviceClient = serviceClientFactory.makeClient( @@ -69,7 +70,7 @@ public DataServerClient( fixedSetServiceLocator, StandardRetryPolicy.noRetries() ); - this.smileMapper = smileMapper; + this.objectMapper = objectMapper; this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } @@ -78,10 +79,17 @@ public Sequence run(Query query, ResponseContext responseContext, JavaType final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); + RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, basePath); + final boolean isSmile = objectMapper.getFactory() instanceof SmileFactory; + if (isSmile) { + requestBuilder = requestBuilder.smileContent(objectMapper, query); + } else { + requestBuilder = requestBuilder.jsonContent(objectMapper, query); + } + ListenableFuture resultStreamFuture = serviceClient.asyncRequest( - new RequestBuilder(HttpMethod.POST, basePath) - .smileContent(smileMapper, query), - new DataServerResponseHandler(query, responseContext, smileMapper) + requestBuilder, + new DataServerResponseHandler(query, responseContext, objectMapper) ); Futures.addCallback( @@ -117,7 +125,7 @@ public JsonParserIterator make() basePath, query, "", - smileMapper + objectMapper ); } From 8d7d6a9a33f7b679129eadf1890b22b65dad2f8e Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 27 Sep 2023 18:51:32 +0530 Subject: [PATCH 11/37] Change realtime check --- .../apache/druid/msq/exec/ControllerImpl.java | 18 +++++- .../input/table/DataSegmentWithLocation.java | 19 +++--- .../input/table/RichSegmentDescriptor.java | 15 +---- .../input/table/SegmentWithDescriptor.java | 8 ++- .../msq/input/table/TableInputSpecSlicer.java | 3 +- .../msq/querykit/BaseLeafFrameProcessor.java | 2 +- .../table/RichSegmentDescriptorTest.java | 9 +-- .../input/table/SegmentsInputSliceTest.java | 3 +- .../input/table/TableInputSpecSlicerTest.java | 63 +++++++------------ 9 files changed, 62 insertions(+), 78 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 1921aa75e2d9..fccb100f4a58 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.util.concurrent.FutureCallback; @@ -189,6 +190,8 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; @@ -1177,8 +1180,19 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() Set unifiedSegmentView = new HashSet<>(metadataStoreSegments); for (ImmutableSegmentLoadInfo segmentLoadInfo : serverViewSegments) { - DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation(segmentLoadInfo.getSegment(), segmentLoadInfo.getServers()); - unifiedSegmentView.add(dataSegmentWithLocation); + ImmutableSet servers = segmentLoadInfo.getServers(); + Set realtimeServerMetadata = servers.stream() + .filter(druidServerMetadata -> + ServerType.REALTIME.equals(druidServerMetadata.getType()) || + ServerType.INDEXER_EXECUTOR.equals(druidServerMetadata.getType())) + .collect(Collectors.toSet()); + if (!realtimeServerMetadata.isEmpty()) { + DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation( + segmentLoadInfo.getSegment(), + realtimeServerMetadata + ); + unifiedSegmentView.add(dataSegmentWithLocation); + } } if (unifiedSegmentView.isEmpty()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java index df937611309c..9c25239e92a0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import org.apache.druid.jackson.CommaListJoinDeserializer; @@ -76,7 +75,17 @@ public DataSegmentWithLocation( Set servers ) { - super(dataSegment.getDataSource(), dataSegment.getInterval(), dataSegment.getVersion(), dataSegment.getLoadSpec(), dataSegment.getDimensions(), dataSegment.getMetrics(), dataSegment.getShardSpec(), dataSegment.getBinaryVersion(), dataSegment.getSize()); + super( + dataSegment.getDataSource(), + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getLoadSpec(), + dataSegment.getDimensions(), + dataSegment.getMetrics(), + dataSegment.getShardSpec(), + dataSegment.getBinaryVersion(), + dataSegment.getSize() + ); this.servers = servers; } @@ -85,10 +94,4 @@ public Set getServers() { return servers; } - - @JsonIgnore - public boolean isRealtime() - { - return getLoadSpec() == null || getLoadSpec().isEmpty(); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index f482bdc581d9..83a396b500fa 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -47,21 +47,18 @@ public class RichSegmentDescriptor extends SegmentDescriptor @Nullable private final Interval fullInterval; private final Set servers; - private final Boolean isRealtime; public RichSegmentDescriptor( final Interval fullInterval, final Interval interval, final String version, final int partitionNumber, - final Set servers, - final boolean isRealtime + final Set servers ) { super(interval, version, partitionNumber); this.fullInterval = interval.equals(Preconditions.checkNotNull(fullInterval, "fullInterval")) ? null : fullInterval; this.servers = servers; - this.isRealtime = isRealtime; } @JsonCreator @@ -79,8 +76,7 @@ static RichSegmentDescriptor fromJson( interval, version, partitionNumber, - servers == null ? ImmutableSet.of() : servers, - isRealtime != null && isRealtime + servers == null ? ImmutableSet.of() : servers ); } @@ -92,13 +88,6 @@ public Set getServers() return servers; } - @JsonProperty("isRealtime") - @JsonInclude(JsonInclude.Include.NON_NULL) - public boolean isRealtime() - { - return Boolean.TRUE.equals(isRealtime); - } - public Interval getFullInterval() { return fullInterval == null ? getInterval() : fullInterval; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 4ccd140bb945..b49c709d3016 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -27,6 +27,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; +import org.apache.druid.utils.CollectionUtils; import java.io.IOException; import java.util.Objects; @@ -93,9 +94,12 @@ public SegmentDescriptor getDescriptor() return descriptor; } - public boolean isRealtimeSegment() + public boolean isLoadedOnServer() { - return descriptor instanceof RichSegmentDescriptor && ((RichSegmentDescriptor) descriptor).isRealtime(); + if (!(descriptor instanceof RichSegmentDescriptor)) { + return false; + } + return !CollectionUtils.isNullOrEmpty(((RichSegmentDescriptor) descriptor).getServers()); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index ba2300a0d7a1..91f2e681e1ea 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -167,8 +167,7 @@ public RichSegmentDescriptor toRichSegmentDescriptor() interval, segment.getVersion(), segment.getShardSpec().getPartitionNum(), - segment instanceof DataSegmentWithLocation ? ((DataSegmentWithLocation) segment).getServers() : null, - segment instanceof DataSegmentWithLocation && ((DataSegmentWithLocation) segment).isRealtime() + segment instanceof DataSegmentWithLocation ? ((DataSegmentWithLocation) segment).getServers() : null ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 1ea87f89db0b..1791a31c6b60 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -157,7 +157,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws return ReturnOrAwait.awaitAll(inputChannels.size()); } else if (baseInput.hasSegment()) { SegmentWithDescriptor segment = baseInput.getSegment(); - if (segment.isRealtimeSegment()) { + if (segment.isLoadedOnServer()) { return runWithLoadedSegment(baseInput.getSegment()); } else { return runWithSegment(baseInput.getSegment()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java index a476416a358d..d1826d6cfd01 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java @@ -38,8 +38,7 @@ public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception Intervals.of("2000/2001"), "2", 3, - null, - false + null ); Assert.assertEquals( @@ -57,8 +56,7 @@ public void testSerdeWithFullIntervalSameAsInterval() throws Exception Intervals.of("2000/2001"), "2", 3, - null, - false + null ); Assert.assertEquals( @@ -76,8 +74,7 @@ public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exc Intervals.of("2000/2001"), "2", 3, - null, - false + null ); Assert.assertEquals( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index d038a6976d10..346f3e15ebf2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -45,8 +45,7 @@ public void testSerde() throws Exception Intervals.of("2000/P1M"), "1", 0, - null, - false + null ) ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java index d4660ebde79a..fd5db7e75f64 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java @@ -147,32 +147,28 @@ public void test_sliceStatic_intervalFilter() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT1.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -215,8 +211,7 @@ public void test_sliceStatic_dimFilter() SEGMENT1.getInterval(), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -248,8 +243,7 @@ public void test_sliceStatic_intervalAndDimFilter() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -261,8 +255,7 @@ public void test_sliceStatic_intervalAndDimFilter() Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -285,16 +278,14 @@ public void test_sliceStatic_oneSlice() SEGMENT1.getInterval(), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -317,8 +308,7 @@ public void test_sliceStatic_needTwoSlices() SEGMENT1.getInterval(), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -330,8 +320,7 @@ public void test_sliceStatic_needTwoSlices() SEGMENT2.getInterval(), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -354,8 +343,7 @@ public void test_sliceStatic_threeSlices() SEGMENT1.getInterval(), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -367,8 +355,7 @@ public void test_sliceStatic_threeSlices() SEGMENT2.getInterval(), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -412,16 +399,14 @@ public void test_sliceDynamic_maxOneSlice() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -449,16 +434,14 @@ public void test_sliceDynamic_needOne() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -486,8 +469,7 @@ public void test_sliceDynamic_needTwoDueToFiles() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -499,8 +481,7 @@ public void test_sliceDynamic_needTwoDueToFiles() Intervals.of("2000/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) @@ -528,8 +509,7 @@ public void test_sliceDynamic_needTwoDueToBytes() Intervals.of("2000/P1M"), SEGMENT1.getVersion(), SEGMENT1.getShardSpec().getPartitionNum(), - null, - false + null ) ) ), @@ -541,8 +521,7 @@ public void test_sliceDynamic_needTwoDueToBytes() Intervals.of("2000/P1M"), SEGMENT2.getVersion(), SEGMENT2.getShardSpec().getPartitionNum(), - null, - false + null ) ) ) From 92c6d75d1db5fbfb87b0a2840e6f671b1837bd80 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 28 Sep 2023 16:31:01 +0530 Subject: [PATCH 12/37] Add logs --- .../org/apache/druid/msq/exec/ControllerImpl.java | 11 +++++++++++ .../msq/exec/LoadedSegmentDataProviderImpl.java | 13 ++++++++++++- .../groupby/GroupByPreShuffleFrameProcessor.java | 3 +++ .../msq/querykit/scan/ScanQueryFrameProcessor.java | 3 +++ .../apache/druid/discovery/DataServerClient.java | 4 ++-- .../druid/discovery/DataServerResponseHandler.java | 3 +++ 6 files changed, 34 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index fccb100f4a58..649e165d1d6f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1178,6 +1178,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() final Collection metadataStoreSegments = FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true); + int realtimeCount = 0; Set unifiedSegmentView = new HashSet<>(metadataStoreSegments); for (ImmutableSegmentLoadInfo segmentLoadInfo : serverViewSegments) { ImmutableSet servers = segmentLoadInfo.getServers(); @@ -1187,6 +1188,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() ServerType.INDEXER_EXECUTOR.equals(druidServerMetadata.getType())) .collect(Collectors.toSet()); if (!realtimeServerMetadata.isEmpty()) { + realtimeCount += 1; DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation( segmentLoadInfo.getSegment(), realtimeServerMetadata @@ -1195,6 +1197,15 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() } } + if (isIncludeRealtime) { + log.info( + "Fetched total [%d] segments from coordinator: [%d] from metadata stoure, [%d] from server view", + unifiedSegmentView.size(), + metadataStoreSegments.size(), + realtimeCount + ); + } + if (unifiedSegmentView.isEmpty()) { return Optional.empty(); } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 5e8fdfc2f9f7..7e72e886a1d2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.query.Queries; @@ -40,13 +41,16 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.rpc.FixedSetServiceLocator; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.server.coordination.DruidServerMetadata; import java.io.IOException; import java.util.List; +import java.util.Set; import java.util.function.Function; public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider { + private static final Logger log = new Logger(LoadedSegmentDataProviderImpl.class); private static final int DEFAULT_NUM_TRIES = 5; private final RichSegmentDescriptor segmentDescriptor; private final String dataSource; @@ -84,14 +88,17 @@ public Pair> ImmutableList.of(segmentDescriptor) ); + Set servers = segmentDescriptor.getServers(); final DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, - new FixedSetServiceLocator(segmentDescriptor.getServers()), + new FixedSetServiceLocator(servers), objectMapper ); final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); + + log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); final ResponseContext responseContext = new DefaultResponseContext(); Pair> statusSequencePair; @@ -101,13 +108,16 @@ public Pair> Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); final List missingSegments = getMissingSegments(responseContext); if (missingSegments.isEmpty()) { + log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); // Segment was found return Pair.of(DataServerQueryStatus.SUCCESS, sequence); } else { Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); if (Boolean.TRUE.equals(wasHandedOff)) { + log.debug("Segment[%s] was handed off.", segmentDescriptor); return Pair.of(DataServerQueryStatus.HANDOFF, null); } else { + log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor); throw new ISE( "Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor @@ -129,6 +139,7 @@ public Pair> ); } catch (Exception e) { + log.error("Exception while fetching rows from dataservers."); throw new IOE(e, "Exception while fetching rows from dataservers."); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index a3ed5bab09d2..605849e7f535 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; @@ -65,6 +66,7 @@ */ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor { + private static final Logger log = new Logger(GroupByPreShuffleFrameProcessor.class); private final GroupByQuery query; private final GroupingEngine groupingEngine; private final ColumnSelectorFactory frameWriterColumnSelectorFactory; @@ -118,6 +120,7 @@ protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment Pair> statusSequencePair = segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { + log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); return runWithSegment(segment); } resultYielder = Yielders.each(statusSequencePair.rhs); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 1ff4467a20bc..f45b33b29990 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -49,6 +49,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.msq.input.ParseExceptionUtils; import org.apache.druid.msq.input.ReadableInput; @@ -88,6 +89,7 @@ */ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor { + private static final Logger log = new Logger(ScanQueryFrameProcessor.class); private final ScanQuery query; private final AtomicLong runningCountForLimit; private final ObjectMapper jsonMapper; @@ -183,6 +185,7 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s ScanResultValue.class ); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { + log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); return runWithSegment(segment); } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 8bed17ef9bb1..6918d2291cab 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -54,7 +54,6 @@ public class DataServerClient { private static final Logger log = new Logger(DataServerClient.class); - private static final String SERVED_SEGMENT_CLIENT_NAME = "ServedSegmentClient"; private final ServiceClient serviceClient; private final ObjectMapper objectMapper; private final ScheduledExecutorService queryCancellationExecutor; @@ -66,7 +65,7 @@ public DataServerClient( ) { serviceClient = serviceClientFactory.makeClient( - SERVED_SEGMENT_CLIENT_NAME, + NodeRole.INDEXER.getJsonName(), fixedSetServiceLocator, StandardRetryPolicy.noRetries() ); @@ -87,6 +86,7 @@ public Sequence run(Query query, ResponseContext responseContext, JavaType requestBuilder = requestBuilder.jsonContent(objectMapper, query); } + log.debug("Sending request to servers for query[%s], request[%s]", query.getId(), requestBuilder.toString()); ListenableFuture resultStreamFuture = serviceClient.asyncRequest( requestBuilder, new DataServerResponseHandler(query, responseContext, objectMapper) diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index dcf722a3001c..6c8968cd87e9 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -20,6 +20,7 @@ package org.apache.druid.discovery; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream; import org.apache.druid.java.util.http.client.response.ClientResponse; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; @@ -35,6 +36,7 @@ public class DataServerResponseHandler implements HttpResponseHandler { + private static final Logger log = new Logger(DataServerResponseHandler.class); private final String queryId; private final ResponseContext responseContext; private final ObjectMapper objectMapper; @@ -49,6 +51,7 @@ public DataServerResponseHandler(Query query, ResponseContext responseCon @Override public ClientResponse handleResponse(HttpResponse response, TrafficCop trafficCop) { + log.debug("Received response status[%s] for queryId[%s]", response.getStatus(), queryId); AppendableByteArrayInputStream in = new AppendableByteArrayInputStream(); in.add(getContentBytes(response.getContent())); From 6bfa3a6462fc05e322e7b1803ba8a849596905fd Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 28 Sep 2023 17:37:52 +0530 Subject: [PATCH 13/37] Fix checkstyles --- .../druid/msq/input/table/RichSegmentDescriptor.java | 3 +-- .../main/java/org/apache/druid/rpc/ServiceLocation.java | 8 +++++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index 83a396b500fa..e184e925d0a1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -67,8 +67,7 @@ static RichSegmentDescriptor fromJson( @JsonProperty("itvl") final Interval interval, @JsonProperty("ver") final String version, @JsonProperty("part") final int partitionNumber, - @JsonProperty("servers") final Set servers, - @JsonProperty("isRealtime") final Boolean isRealtime + @JsonProperty("servers") final Set servers ) { return new RichSegmentDescriptor( diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java index a43e3727271b..e84c5e318544 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -80,7 +81,12 @@ private static int getPortFromString(String s) } Iterator iterator = SPLITTER.split(s).iterator(); ImmutableList strings = ImmutableList.copyOf(iterator); - return Integer.parseInt(strings.get(1)); + try { + return Integer.parseInt(strings.get(1)); + } + catch (NumberFormatException e) { + throw new ISE(e, "Unable to parse port out of %s", strings.get(1)); + } } public String getHost() From 065b7fdaabfdd6fc5fe6ea1770e0f07386f6da0a Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 28 Sep 2023 17:44:39 +0530 Subject: [PATCH 14/37] Change return to yielder --- .../msq/exec/LoadedSegmentDataProvider.java | 7 +++-- .../exec/LoadedSegmentDataProviderImpl.java | 27 +++++++++------- .../input/table/SegmentWithDescriptor.java | 9 ++++-- .../GroupByPreShuffleFrameProcessor.java | 6 ++-- .../scan/ScanQueryFrameProcessor.java | 7 +++-- .../druid/query/IterableRowsCursorHelper.java | 31 +++++++++++++++++++ 6 files changed, 64 insertions(+), 23 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 98282d5a6c9f..5c66ff26eb3a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -21,6 +21,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; import java.io.IOException; @@ -28,10 +30,11 @@ public interface LoadedSegmentDataProvider { - Pair> fetchRowsFromDataServer( + Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Class queryResultType + Class queryResultType, + Closer closer ) throws IOException; enum DataServerQueryStatus diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 7e72e886a1d2..22ae464ce4b7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -30,6 +30,9 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.RichSegmentDescriptor; @@ -77,10 +80,11 @@ public LoadedSegmentDataProviderImpl( } @Override - public Pair> fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Class resultClass + Class resultClass, + Closer closer ) throws IOException { final Query preparedQuery = Queries.withSpecificSegments( @@ -101,16 +105,22 @@ public Pair> log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); final ResponseContext responseContext = new DefaultResponseContext(); - Pair> statusSequencePair; + Pair> statusSequencePair; try { statusSequencePair = RetryUtils.retry( () -> { Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); + Yielder yielder = Yielders.each(mappingFunction.apply(sequence) + .map(row -> { + channelCounters.incrementRowCount(); + return row; + })); + closer.register(yielder); final List missingSegments = getMissingSegments(responseContext); if (missingSegments.isEmpty()) { log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); // Segment was found - return Pair.of(DataServerQueryStatus.SUCCESS, sequence); + return Pair.of(DataServerQueryStatus.SUCCESS, yielder); } else { Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); if (Boolean.TRUE.equals(wasHandedOff)) { @@ -129,14 +139,7 @@ public Pair> numRetriesOnMissingSegments ); - return Pair.of( - statusSequencePair.lhs, - mappingFunction.apply(statusSequencePair.rhs) - .map(row -> { - channelCounters.incrementRowCount(); - return row; - }) - ); + return statusSequencePair; } catch (Exception e) { log.error("Exception while fetching rows from dataservers."); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index b49c709d3016..1fc9447c3da1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -23,6 +23,8 @@ import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.query.Query; import org.apache.druid.query.SegmentDescriptor; @@ -77,13 +79,14 @@ public ResourceHolder getOrLoad() return segmentSupplier.get(); } - public Pair> fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Class queryResultType + Class queryResultType, + Closer closer ) throws IOException { - return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, queryResultType); + return loadedSegmentDataProvider.fetchRowsFromDataServer(query, mappingFunction, queryResultType, closer); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 605849e7f535..5bb9f68b75a8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -117,13 +117,13 @@ private GroupByQuery prepareGroupByQuery(GroupByQuery query) protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { - Pair> statusSequencePair = - segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class); + Pair> statusSequencePair = + segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class, closer); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); return runWithSegment(segment); } - resultYielder = Yielders.each(statusSequencePair.rhs); + resultYielder = statusSequencePair.rhs; } populateFrameWriterAndFlushIfNeeded(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index f45b33b29990..e86e31358a2a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -178,11 +178,12 @@ public static Sequence mappingFunction(Sequence input protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { - final Pair> statusSequencePair = + final Pair> statusSequencePair = segment.fetchRowsFromDataServer( query, ScanQueryFrameProcessor::mappingFunction, - ScanResultValue.class + ScanResultValue.class, + closer ); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); @@ -190,7 +191,7 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s } RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); - RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromSequence( + RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( statusSequencePair.rhs, rowSignature ); diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java index c5bb271213e8..15c36155ccb7 100644 --- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java +++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java @@ -103,4 +103,35 @@ public Object[] next() rowSignature ); } + + public static RowBasedCursor getCursorFromYielder(Yielder yielderParam, RowSignature rowSignature) + { + return getCursorFromIterable( + new Iterable() + { + Yielder yielder = yielderParam; + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return !yielder.isDone(); + } + + @Override + public Object[] next() + { + Object[] retVal = yielder.get(); + yielder = yielder.next(null); + return retVal; + } + }; + } + }, + rowSignature + ); + } } From f9a05d0a5b663ee241dc0153e0e2d7d213aa4cad Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 29 Sep 2023 21:01:21 +0530 Subject: [PATCH 15/37] Fix NPE --- .../msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java | 3 ++- .../druid/msq/querykit/scan/ScanQueryFrameProcessor.java | 3 ++- server/src/main/java/org/apache/druid/rpc/ServiceLocation.java | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 5bb9f68b75a8..ec9063996a30 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -120,7 +120,8 @@ protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment Pair> statusSequencePair = segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class, closer); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { - log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); + log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", + segment.getDescriptor()); return runWithSegment(segment); } resultYielder = statusSequencePair.rhs; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index e86e31358a2a..b5281facd146 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -186,7 +186,8 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s closer ); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { - log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment); + log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", + segment.getDescriptor()); return runWithSegment(segment); } diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java index e84c5e318544..6e20cdca6076 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -57,7 +57,7 @@ public static ServiceLocation fromDruidNode(final DruidNode druidNode) public static ServiceLocation fromDruidServerMetadata(final DruidServerMetadata druidServerMetadata) { - final String host = getHostFromString(druidServerMetadata.getHostAndPort()); + final String host = getHostFromString(druidServerMetadata.getHost()); int plaintextPort = getPortFromString(druidServerMetadata.getHostAndPort()); int tlsPort = getPortFromString(druidServerMetadata.getHostAndTlsPort()); return new ServiceLocation(host, plaintextPort, tlsPort, ""); From 3ea2b43574d7beced0040a7a3d07dc9284f163c2 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 1 Oct 2023 11:12:58 +0530 Subject: [PATCH 16/37] Add unit tests --- .../query/IterableRowsCursorHelperTest.java | 8 ++ .../coordination/DruidServerMetadata.java | 4 +- .../CoordinatorClientImplTest.java | 78 ++++++++++++++++++- .../apache/druid/rpc/ServiceLocationTest.java | 41 ++++++++++ 4 files changed, 129 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java index 1acaceabbd60..2f423ca9d435 100644 --- a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java +++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.column.ColumnType; @@ -60,6 +61,13 @@ public void getCursorFromSequence() testCursorMatchesRowSequence(cursor, rowSignature, rows); } + @Test + public void getCursorFromYielder() + { + Cursor cursor = IterableRowsCursorHelper.getCursorFromYielder(Yielders.each(Sequences.simple(rows)), rowSignature); + testCursorMatchesRowSequence(cursor, rowSignature, rows); + } + private void testCursorMatchesRowSequence( Cursor cursor, RowSignature expectedRowSignature, diff --git a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java index 3fda41b08dab..1a8a4388f8d8 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java @@ -31,6 +31,7 @@ public class DruidServerMetadata { private final String name; + @Nullable private final String hostAndPort; @Nullable private final String hostAndTlsPort; @@ -42,7 +43,7 @@ public class DruidServerMetadata @JsonCreator public DruidServerMetadata( @JsonProperty("name") String name, - @JsonProperty("host") String hostAndPort, + @JsonProperty("host") @Nullable String hostAndPort, @JsonProperty("hostAndTlsPort") @Nullable String hostAndTlsPort, @JsonProperty("maxSize") long maxSize, @JsonProperty("type") ServerType type, @@ -70,6 +71,7 @@ public String getHost() return getHostAndTlsPort() != null ? getHostAndTlsPort() : getHostAndPort(); } + @Nullable @JsonProperty("host") public String getHostAndPort() { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index f48e21327a0b..8977d64ee555 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -21,13 +21,18 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -42,6 +47,7 @@ import javax.ws.rs.core.MediaType; import java.util.Collections; import java.util.List; +import java.util.Set; public class CoordinatorClientImplTest { @@ -130,7 +136,10 @@ public void test_fetchSegment() throws Exception .build(); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/metadata/datasources/xyz/segments/def?includeUnused=true"), + new RequestBuilder( + HttpMethod.GET, + "/druid/coordinator/v1/metadata/datasources/xyz/segments/def?includeUnused=true" + ), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), jsonMapper.writeValueAsBytes(segment) @@ -168,4 +177,71 @@ public void test_fetchUsedSegments() throws Exception coordinatorClient.fetchUsedSegments("xyz", intervals).get() ); } + + @Test + public void test_fetchServerViewSegments() throws Exception + { + + final List intervals = ImmutableList.of( + Intervals.of("2001/2002"), + Intervals.of("2501/2502") + ); + + final Set serverMetadataSet = + ImmutableSet.of( + new DruidServerMetadata( + "TEST_SERVER", + "testhost:9092", + null, + 1, + ServerType.INDEXER_EXECUTOR, + "tier1", + 0 + ) + ); + + final ImmutableSegmentLoadInfo immutableSegmentLoadInfo1 = new ImmutableSegmentLoadInfo( + DataSegment.builder() + .dataSource("xyz") + .interval(intervals.get(0)) + .version("1") + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(), + serverMetadataSet + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/datasources/xyz/intervals/2001-01-01T00:00:00.000Z_2002-01-01T00:00:00.000Z/serverview?full"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(Collections.singletonList(immutableSegmentLoadInfo1)) + ); + + final ImmutableSegmentLoadInfo immutableSegmentLoadInfo2 = new ImmutableSegmentLoadInfo( + DataSegment.builder() + .dataSource("xyz") + .interval(intervals.get(1)) + .version("1") + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(), + serverMetadataSet + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/datasources/xyz/intervals/2501-01-01T00:00:00.000Z_2502-01-01T00:00:00.000Z/serverview?full"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(Collections.singletonList(immutableSegmentLoadInfo2)) + ); + + List segmentLoadInfoList = + ImmutableList.of(immutableSegmentLoadInfo1, immutableSegmentLoadInfo2); + + Assert.assertEquals( + segmentLoadInfoList, + coordinatorClient.fetchServerViewSegments("xyz", intervals) + ); + } } diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java index 3fba0c409e08..6aec0e2b6060 100644 --- a/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java +++ b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java @@ -20,10 +20,51 @@ package org.apache.druid.rpc; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.junit.Assert; import org.junit.Test; public class ServiceLocationTest { + @Test + public void test_fromDruidServerMetadata_withPort() + { + DruidServerMetadata druidServerMetadata = new DruidServerMetadata( + "name", + "hostName:9092", + null, + 1, + ServerType.INDEXER_EXECUTOR, + "tier1", + 2 + ); + + Assert.assertEquals( + new ServiceLocation("hostName", 9092, -1, ""), + ServiceLocation.fromDruidServerMetadata(druidServerMetadata) + ); + } + + @Test + public void test_fromDruidServerMetadata_withTlsPort() + { + DruidServerMetadata druidServerMetadata = new DruidServerMetadata( + "name", + null, + "hostName:8100", + 1, + ServerType.INDEXER_EXECUTOR, + "tier1", + 2 + ); + + Assert.assertEquals( + new ServiceLocation("hostName", -1, 8100, ""), + ServiceLocation.fromDruidServerMetadata(druidServerMetadata) + ); + } + @Test public void test_equals() { From 763fea23d1e5262f08508a655fe2d3b4b6cd14fd Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 1 Oct 2023 15:06:34 +0530 Subject: [PATCH 17/37] Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 18 +++++++++++------- .../msq/exec/LoadedSegmentDataProvider.java | 2 ++ .../exec/LoadedSegmentDataProviderFactory.java | 3 --- .../exec/LoadedSegmentDataProviderImpl.java | 4 +--- .../external/ExternalInputSliceReader.java | 3 ++- .../input/inline/InlineInputSliceReader.java | 5 +++-- .../input/lookup/LookupInputSliceReader.java | 3 ++- .../msq/input/table/RichSegmentDescriptor.java | 14 +++++++++++++- .../msq/input/table/SegmentWithDescriptor.java | 11 ++++------- .../input/table/SegmentsInputSliceReader.java | 2 +- 10 files changed, 39 insertions(+), 26 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 649e165d1d6f..5dd35e260030 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1166,22 +1166,26 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() final boolean isIncludeRealtime = MultiStageQueryContext.isIncludeRealtime(task.getQuerySpec().getQuery().context()); return (dataSource, intervals) -> { - final Iterable serverViewSegments; + final Iterable realtimeAndHistoricalSegments; // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two calls. + // Segments loaded on historicals are also returned here, we dedup it below. if (isIncludeRealtime) { - serverViewSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); + realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); } else { - serverViewSegments = ImmutableList.of(); + realtimeAndHistoricalSegments = ImmutableList.of(); } - final Collection metadataStoreSegments = + // Fetch all published used segments from the metadata store. + final Collection publishedUsedSegments = FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true); int realtimeCount = 0; - Set unifiedSegmentView = new HashSet<>(metadataStoreSegments); - for (ImmutableSegmentLoadInfo segmentLoadInfo : serverViewSegments) { + Set unifiedSegmentView = new HashSet<>(publishedUsedSegments); + for (ImmutableSegmentLoadInfo segmentLoadInfo : realtimeAndHistoricalSegments) { ImmutableSet servers = segmentLoadInfo.getServers(); + // Filter out only realtime servers. We don't want to query historicals for now, but we can in the future. + // This check can be modified then. Set realtimeServerMetadata = servers.stream() .filter(druidServerMetadata -> ServerType.REALTIME.equals(druidServerMetadata.getType()) || @@ -1201,7 +1205,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() log.info( "Fetched total [%d] segments from coordinator: [%d] from metadata stoure, [%d] from server view", unifiedSegmentView.size(), - metadataStoreSegments.size(), + publishedUsedSegments.size(), realtimeCount ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 5c66ff26eb3a..f0564ca7589c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.query.Query; import java.io.IOException; @@ -32,6 +33,7 @@ public interface LoadedSegmentDataProvider { Pair> fetchRowsFromDataServer( Query query, + RichSegmentDescriptor segmentDescriptor, Function, Sequence> mappingFunction, Class queryResultType, Closer closer diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 7bfd4c8d1612..2ca2d636bf45 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.rpc.ServiceClientFactory; public class LoadedSegmentDataProviderFactory @@ -44,13 +43,11 @@ public LoadedSegmentDataProviderFactory( } public LoadedSegmentDataProvider createLoadedSegmentDataProvider( - RichSegmentDescriptor segmentDescriptor, String dataSource, ChannelCounters channelCounters ) { return new LoadedSegmentDataProviderImpl( - segmentDescriptor, dataSource, channelCounters, serviceClientFactory, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 22ae464ce4b7..e8207c5f1b71 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -55,7 +55,6 @@ public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider { private static final Logger log = new Logger(LoadedSegmentDataProviderImpl.class); private static final int DEFAULT_NUM_TRIES = 5; - private final RichSegmentDescriptor segmentDescriptor; private final String dataSource; private final ChannelCounters channelCounters; private final ServiceClientFactory serviceClientFactory; @@ -63,7 +62,6 @@ public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider private final ObjectMapper objectMapper; public LoadedSegmentDataProviderImpl( - RichSegmentDescriptor segmentDescriptor, String dataSource, ChannelCounters channelCounters, ServiceClientFactory serviceClientFactory, @@ -71,7 +69,6 @@ public LoadedSegmentDataProviderImpl( ObjectMapper objectMapper ) { - this.segmentDescriptor = segmentDescriptor; this.dataSource = dataSource; this.channelCounters = channelCounters; this.serviceClientFactory = serviceClientFactory; @@ -82,6 +79,7 @@ public LoadedSegmentDataProviderImpl( @Override public Pair> fetchRowsFromDataServer( Query query, + RichSegmentDescriptor segmentDescriptor, Function, Sequence> mappingFunction, Class resultClass, Closer closer diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 7f4cd6ffb895..084d58e217d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -42,6 +42,7 @@ import org.apache.druid.msq.input.NilInputSource; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.segment.RowBasedSegment; @@ -160,7 +161,7 @@ private static Iterator inputSourceSegmentIterator( return new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), null, - segmentId.toDescriptor() + new RichSegmentDescriptor(segmentId.toDescriptor(), null, null) ); } ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 40d4a2237382..25f06c7cd40c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -27,9 +27,9 @@ import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.InlineDataSource; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.timeline.SegmentId; @@ -43,7 +43,8 @@ public class InlineInputSliceReader implements InputSliceReader { public static final String SEGMENT_ID = "__inline"; - private static final SegmentDescriptor DUMMY_SEGMENT_DESCRIPTOR = SegmentId.dummy(SEGMENT_ID).toDescriptor(); + private static final RichSegmentDescriptor DUMMY_SEGMENT_DESCRIPTOR + = new RichSegmentDescriptor(SegmentId.dummy(SEGMENT_ID).toDescriptor(), null, null); private final SegmentWrangler segmentWrangler; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index ecf0c0d70cf0..b601b043ac13 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -29,6 +29,7 @@ import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.LookupDataSource; import org.apache.druid.segment.Segment; @@ -100,7 +101,7 @@ public ReadableInputs attach( return ResourceHolder.fromCloseable(segment); }, null, - SegmentId.dummy(lookupName).toDescriptor() + new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null, null) ) ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index e184e925d0a1..b99b50abf791 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -46,6 +46,7 @@ public class RichSegmentDescriptor extends SegmentDescriptor { @Nullable private final Interval fullInterval; + @Nullable private final Set servers; public RichSegmentDescriptor( @@ -61,13 +62,24 @@ public RichSegmentDescriptor( this.servers = servers; } + public RichSegmentDescriptor( + SegmentDescriptor segmentDescriptor, + @Nullable Interval fullInterval, + @Nullable Set servers + ) + { + super(segmentDescriptor.getInterval(), segmentDescriptor.getVersion(), segmentDescriptor.getPartitionNumber()); + this.fullInterval = fullInterval; + this.servers = servers; + } + @JsonCreator static RichSegmentDescriptor fromJson( @JsonProperty("fi") @Nullable final Interval fullInterval, @JsonProperty("itvl") final Interval interval, @JsonProperty("ver") final String version, @JsonProperty("part") final int partitionNumber, - @JsonProperty("servers") final Set servers + @JsonProperty("servers") @Nullable final Set servers ) { return new RichSegmentDescriptor( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 1fc9447c3da1..c598de64fdb8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -43,7 +43,7 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; private final LoadedSegmentDataProvider loadedSegmentDataProvider; - private final SegmentDescriptor descriptor; + private final RichSegmentDescriptor descriptor; /** * Create a new instance. @@ -57,7 +57,7 @@ public class SegmentWithDescriptor public SegmentWithDescriptor( final Supplier> segmentSupplier, final LoadedSegmentDataProvider loadedSegmentDataProvider, - final SegmentDescriptor descriptor + final RichSegmentDescriptor descriptor ) { this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment"); @@ -86,7 +86,7 @@ public Pair dataSegmentIterator( return new SegmentWithDescriptor( dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), - loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(descriptor, dataSource, channelCounters), + loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, channelCounters), descriptor ); } From 9c9e099b1f4b98ae2eaa99a3855cb45a675607e8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 1 Oct 2023 20:26:33 +0530 Subject: [PATCH 18/37] Add unit tests --- .../druid/msq/exec/MSQLoadedSegmentTests.java | 309 ++++++++++++++++++ .../druid/msq/test/CalciteMSQTestsHelper.java | 14 + .../test/CalciteSelectJoinQueryMSQTest.java | 4 +- .../msq/test/CalciteSelectQueryMSQTest.java | 3 +- .../apache/druid/msq/test/MSQTestBase.java | 21 +- .../msq/test/MSQTestControllerContext.java | 17 +- .../test/MSQTestOverlordServiceClient.java | 18 +- .../druid/msq/test/MSQTestWorkerContext.java | 3 +- 8 files changed, 378 insertions(+), 11 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java new file mode 100644 index 000000000000..fb43ee839f8f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.ImmutableSegmentLoadInfo; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.planner.ColumnMapping; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.hamcrest.CoreMatchers; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Map; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; + +public class MSQLoadedSegmentTests extends MSQTestBase +{ + public static final Map REALTIME_QUERY_CTX = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_INCLUDE_REALTIME, true) + .build(); + public static final DataSegment LOADED_SEGMENT_1 = + DataSegment.builder() + .dataSource(CalciteTests.DATASOURCE1) + .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( + "TestDataServer", + "hostName:9092", + null, + 2, + ServerType.REALTIME, + "tier1", + 2 + ); + + @Before + public void setUp() throws Exception + { + loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); + } + + @Test + public void testSelectWithLoadedSegmentsOnFoo() throws IOException + { + RowSignature resultSignature = RowSignature.builder() + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .build(); + + doReturn( + Pair.of( + LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, + Yielders.each( + Sequences.simple( + ImmutableList.of( + new Object[]{"5", "qwe"}, + new Object[]{"123", "tyu"} + ) + ) + ) + ) + ) + .when(loadedSegmentDataProvider) + .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + + testSelectQuery() + .setSql("select dim1, dim2 from foo") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2") + .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{"", "a"}, + new Object[]{"5", "qwe"}, + new Object[]{"10.1", null}, + new Object[]{"123", "tyu"}, + new Object[]{"2", ""}, + new Object[]{"1", "a"}, + new Object[]{"def", "abc"}, + new Object[]{"abc", null} + )) + .verifyResults(); + } + + @Test + public void testGroupByWithLoadedSegmentsOnFoo() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doReturn( + Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, + Yielders.each( + Sequences.simple( + ImmutableList.of( + ResultRow.of(1L, 2L) + ) + ) + ) + ) + ) + .when(loadedSegmentDataProvider) + .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration + .eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L})) + .verifyResults(); + } + + @Test + public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doReturn( + Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, + Yielders.each( + Sequences.simple( + ImmutableList.of( + ResultRow.of(1L, 2L))))) + ).when(loadedSegmentDataProvider) + .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L})) + .verifyResults(); + } + + @Test + public void testDataServerQueryFailedShouldFail() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doThrow( + new ISE("Segment could not be found on data server, but segment was not handed off.") + ) + .when(loadedSegmentDataProvider) + .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + doReturn(Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, null)); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class)) + .verifyExecutionError(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 0c840b14a88e..c86f831a33e7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.querykit.DataSegmentProvider; @@ -78,6 +79,7 @@ import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.File; @@ -93,6 +95,9 @@ import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doThrow; /** * Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests @@ -162,6 +167,7 @@ public String getFormatString() binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); + binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( @@ -179,6 +185,14 @@ public String getFormatString() ); } + private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() + { + LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); + doThrow(new AssertionError("Test does not support loaded segment query")) + .when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); + return mockFactory; + } + private static Supplier> getSupplierForSegment(SegmentId segmentId) { final TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java index ab7b1ed7d7cf..114583d31a1a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.test; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import com.google.inject.Module; import org.apache.calcite.rel.RelRoot; @@ -128,7 +129,8 @@ public SqlEngine createEngine( queryJsonMapper, injector, new MSQTestTaskActionClient(queryJsonMapper), - workerMemoryParameters + workerMemoryParameters, + ImmutableList.of() ); return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java index d7131d0b7ef8..b3d1104353ca 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java @@ -85,7 +85,8 @@ public SqlEngine createEngine( queryJsonMapper, injector, new MSQTestTaskActionClient(queryJsonMapper), - workerMemoryParameters + workerMemoryParameters, + ImmutableList.of() ); return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index b9037a949c45..9d4ec1045d81 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -36,6 +36,7 @@ import com.google.inject.TypeLiteral; import com.google.inject.util.Modules; import com.google.inject.util.Providers; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.config.NullHandling; @@ -83,6 +84,8 @@ import org.apache.druid.msq.counters.QueryCounterSnapshot; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.LoadedSegmentDataProvider; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; @@ -292,6 +295,11 @@ public class MSQTestBase extends BaseCalciteQueryTest protected AuthorizerMapper authorizerMapper; private IndexIO indexIO; + // Contains the metadata of loaded segments + protected List loadedSegmentsMetadata = new ArrayList<>(); + // Mocks the return of data from data servers + protected LoadedSegmentDataProvider loadedSegmentDataProvider = mock(LoadedSegmentDataProvider.class); + private MSQTestSegmentManager segmentManager; private SegmentCacheManager segmentCacheManager; @Rule @@ -415,6 +423,7 @@ public String getFormatString() .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); + binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); @@ -494,7 +503,8 @@ public String getFormatString() objectMapper, injector, testTaskActionClient, - workerMemoryParameters + workerMemoryParameters, + loadedSegmentsMetadata ); CatalogResolver catalogResolver = createMockCatalogResolver(); final InProcessViewManager viewManager = new InProcessViewManager(SqlTestFramework.DRUID_VIEW_MACRO_FACTORY); @@ -567,6 +577,15 @@ protected long[] createExpectedFrameArray(int length, int value) return array; } + private LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() + { + LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); + doReturn(loadedSegmentDataProvider) + .when(mockFactory) + .createLoadedSegmentDataProvider(anyString(), any()); + return mockFactory; + } + @Nonnull private Supplier> getSupplierForSegment(SegmentId segmentId) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 2ee2207fd83a..6be79075165b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -26,6 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Injector; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -54,6 +55,7 @@ import javax.annotation.Nullable; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -91,7 +93,8 @@ public MSQTestControllerContext( ObjectMapper mapper, Injector injector, TaskActionClient taskActionClient, - WorkerMemoryParameters workerMemoryParameters + WorkerMemoryParameters workerMemoryParameters, + List loadedSegments ) { this.mapper = mapper; @@ -112,6 +115,18 @@ public MSQTestControllerContext( .collect(Collectors.toList()) ) ); + + Mockito.when(coordinatorClient.fetchServerViewSegments( + ArgumentMatchers.anyString(), + ArgumentMatchers.any() + ) + ).thenAnswer(invocation -> loadedSegments.stream() + .filter(immutableSegmentLoadInfo -> + immutableSegmentLoadInfo.getSegment() + .getDataSource() + .equals(invocation.getArguments()[0])) + .collect(Collectors.toList()) + ); this.workerMemoryParameters = workerMemoryParameters; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java index 1b49982cad46..c5f601d875ef 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Injector; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; @@ -43,6 +44,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; public class MSQTestOverlordServiceClient extends NoopOverlordClient @@ -51,10 +53,11 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient private final ObjectMapper objectMapper; private final TaskActionClient taskActionClient; private final WorkerMemoryParameters workerMemoryParameters; - private Map inMemoryControllers = new HashMap<>(); - private Map> reports = new HashMap<>(); - private Map inMemoryControllerTask = new HashMap<>(); - private Map inMemoryTaskStatus = new HashMap<>(); + private final List loadedSegmentMetadata; + private final Map inMemoryControllers = new HashMap<>(); + private final Map> reports = new HashMap<>(); + private final Map inMemoryControllerTask = new HashMap<>(); + private final Map inMemoryTaskStatus = new HashMap<>(); public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); public static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); @@ -65,13 +68,15 @@ public MSQTestOverlordServiceClient( ObjectMapper objectMapper, Injector injector, TaskActionClient taskActionClient, - WorkerMemoryParameters workerMemoryParameters + WorkerMemoryParameters workerMemoryParameters, + List loadedSegmentMetadata ) { this.objectMapper = objectMapper; this.injector = injector; this.taskActionClient = taskActionClient; this.workerMemoryParameters = workerMemoryParameters; + this.loadedSegmentMetadata = loadedSegmentMetadata; } @Override @@ -84,7 +89,8 @@ public ListenableFuture runTask(String taskId, Object taskObject) objectMapper, injector, taskActionClient, - workerMemoryParameters + workerMemoryParameters, + loadedSegmentMetadata ); MSQControllerTask cTask = objectMapper.convertValue(taskObject, MSQControllerTask.class); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 26a81ec81758..c35f93357fa3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; @@ -158,7 +159,7 @@ public void setObjectMapper(ObjectMapper objectMapper) ), indexIO, injector.getInstance(DataSegmentProvider.class), - null, + injector.getInstance(LoadedSegmentDataProviderFactory.class), workerMemoryParameters ); } From d5199d777949471ba5b4dc4912bf10f8b46c5762 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 1 Oct 2023 21:38:13 +0530 Subject: [PATCH 19/37] Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 32 ++-- .../msq/exec/LoadedSegmentDataProvider.java | 4 +- .../exec/LoadedSegmentDataProviderImpl.java | 8 +- .../input/table/DataSegmentWithLocation.java | 3 +- .../input/table/SegmentWithDescriptor.java | 4 +- .../msq/util/MultiStageQueryContext.java | 39 ++++- .../LoadedSegmentDataProviderImplTest.java | 138 ++++++++++++++++++ .../druid/msq/exec/MSQLoadedSegmentTests.java | 2 +- 8 files changed, 202 insertions(+), 28 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 4fd45e119e8c..397cf5d1e586 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -191,7 +191,6 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; @@ -1164,14 +1163,21 @@ private QueryKit makeQueryControllerToolKit() private DataSegmentTimelineView makeDataSegmentTimelineView() { - final boolean isIncludeRealtime = MultiStageQueryContext.isIncludeRealtime(task.getQuerySpec().getQuery().context()); + final MultiStageQueryContext.SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources( + task.getQuerySpec() + .getQuery() + .context() + ); + + final boolean includeRealtime + = MultiStageQueryContext.SegmentSource.shouldQueryRealtimeServers(includeSegmentSource); return (dataSource, intervals) -> { final Iterable realtimeAndHistoricalSegments; - // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two calls. - // Segments loaded on historicals are also returned here, we dedup it below. - if (isIncludeRealtime) { + // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two + // calls. Segments loaded on historicals are also returned here, we deduplicate it below. + if (includeRealtime) { realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); } else { realtimeAndHistoricalSegments = ImmutableList.of(); @@ -1182,16 +1188,20 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true); int realtimeCount = 0; + + // Deduplicate segments, giving preference to metadata store segments. We do this so that if any segments have been + // handed off in between the two metadata calls above, we directly fetch it from deep storage. Set unifiedSegmentView = new HashSet<>(publishedUsedSegments); for (ImmutableSegmentLoadInfo segmentLoadInfo : realtimeAndHistoricalSegments) { ImmutableSet servers = segmentLoadInfo.getServers(); // Filter out only realtime servers. We don't want to query historicals for now, but we can in the future. // This check can be modified then. - Set realtimeServerMetadata = servers.stream() - .filter(druidServerMetadata -> - ServerType.REALTIME.equals(druidServerMetadata.getType()) || - ServerType.INDEXER_EXECUTOR.equals(druidServerMetadata.getType())) - .collect(Collectors.toSet()); + Set realtimeServerMetadata + = servers.stream() + .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() + .contains(druidServerMetadata.getType()) + ) + .collect(Collectors.toSet()); if (!realtimeServerMetadata.isEmpty()) { realtimeCount += 1; DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation( @@ -1202,7 +1212,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() } } - if (isIncludeRealtime) { + if (includeRealtime) { log.info( "Fetched total [%d] segments from coordinator: [%d] from metadata stoure, [%d] from server view", unifiedSegmentView.size(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index f0564ca7589c..c904f870a18e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -31,10 +31,10 @@ public interface LoadedSegmentDataProvider { - Pair> fetchRowsFromDataServer( + Pair> fetchRowsFromDataServer( Query query, RichSegmentDescriptor segmentDescriptor, - Function, Sequence> mappingFunction, + Function, Sequence> mappingFunction, Class queryResultType, Closer closer ) throws IOException; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index e8207c5f1b71..85faa1680a71 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -77,10 +77,10 @@ public LoadedSegmentDataProviderImpl( } @Override - public Pair> fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, RichSegmentDescriptor segmentDescriptor, - Function, Sequence> mappingFunction, + Function, Sequence> mappingFunction, Class resultClass, Closer closer ) throws IOException @@ -103,12 +103,12 @@ public Pair> log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); final ResponseContext responseContext = new DefaultResponseContext(); - Pair> statusSequencePair; + Pair> statusSequencePair; try { statusSequencePair = RetryUtils.retry( () -> { Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); - Yielder yielder = Yielders.each(mappingFunction.apply(sequence) + Yielder yielder = Yielders.each(mappingFunction.apply(sequence) .map(row -> { channelCounters.incrementRowCount(); return row; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java index 9c25239e92a0..afc006902e89 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.google.common.base.Preconditions; import org.apache.druid.jackson.CommaListJoinDeserializer; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.CompactionState; @@ -67,7 +68,7 @@ public DataSegmentWithLocation( ) { super(dataSource, interval, version, loadSpec, dimensions, metrics, shardSpec, lastCompactionState, binaryVersion, size, pruneSpecsHolder); - this.servers = servers; + this.servers = Preconditions.checkNotNull(servers, "servers"); } public DataSegmentWithLocation( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index c598de64fdb8..98822051b7e3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -79,9 +79,9 @@ public ResourceHolder getOrLoad() return segmentSupplier.get(); } - public Pair> fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, - Function, Sequence> mappingFunction, + Function, Sequence> mappingFunction, Class queryResultType, Closer closer ) throws IOException diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 6273aa0bd4b4..d7d08e3648fc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; @@ -33,12 +34,14 @@ import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.server.coordination.ServerType; import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -90,8 +93,8 @@ public class MultiStageQueryContext public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations"; private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; - public static final String CTX_INCLUDE_REALTIME = "includeRealtimeResults"; - public static final boolean DEFAULT_INCLUDE_REALTIME = false; + public static final String CTX_INCLUDE_SEGMENT_SOURCE = "includeSegmentSource"; + public static final SegmentSource DEFAULT_INCLUDE_SEGMENT_SOURCE = SegmentSource.NONE; public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false; @@ -184,12 +187,12 @@ public static boolean isFinalizeAggregations(final QueryContext queryContext) ); } - public static boolean isIncludeRealtime(final QueryContext queryContext) + public static SegmentSource getSegmentSources(final QueryContext queryContext) { - return QueryContexts.getAsBoolean( - CTX_INCLUDE_REALTIME, - queryContext.getBoolean(CTX_INCLUDE_REALTIME, DEFAULT_INCLUDE_REALTIME), - DEFAULT_INCLUDE_REALTIME + return queryContext.getEnum( + CTX_INCLUDE_SEGMENT_SOURCE, + SegmentSource.class, + DEFAULT_INCLUDE_SEGMENT_SOURCE ); } @@ -313,4 +316,26 @@ static IndexSpec decodeIndexSpec(@Nullable final Object indexSpecObject, final O throw QueryContexts.badValueException(CTX_INDEX_SPEC, "an indexSpec", indexSpecObject); } } + + public enum SegmentSource { + + NONE(ImmutableSet.of()), + REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR)); + + private final Set usedServerTypes; + + SegmentSource(Set usedServerTypes) + { + this.usedServerTypes = usedServerTypes; + } + + public Set getUsedServerTypes() + { + return usedServerTypes; + } + + public static boolean shouldQueryRealtimeServers(SegmentSource segmentSource) { + return REALTIME.equals(segmentSource); + } + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java new file mode 100644 index 000000000000..716594058faa --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.ServiceLocations; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.io.ByteArrayInputStream; +import java.util.Arrays; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; + +@RunWith(MockitoJUnitRunner.class) +public class LoadedSegmentDataProviderImplTest +{ + private final String TEST_HOST = "indexer"; + private final int TEST_PORT = 9092; + @Mock + ServiceClientFactory serviceClientFactory; + @Mock + ServiceClient serviceClient; + + @Mock + CoordinatorClient coordinatorClient; + final RichSegmentDescriptor testSegmentDescriptor = new RichSegmentDescriptor( + Intervals.ETERNITY, + Intervals.ETERNITY, + "v1", + 1, + ImmutableSet.of(new DruidServerMetadata("indexer", TEST_HOST + ":" + TEST_PORT, null, 1, ServerType.REALTIME, "tier", 2))); + + final LoadedSegmentDataProviderImpl target = new LoadedSegmentDataProviderImpl( + QueryRunnerTestHelper.DATA_SOURCE, new ChannelCounters(), serviceClientFactory, coordinatorClient, new ObjectMapper() + ); + + @Before + public void setUp() throws Exception + { + serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> { + Assert.assertEquals( + FutureUtils.getUnchecked(serviceLocator.locate(), true), + ServiceLocations.forLocation(new ServiceLocation(TEST_HOST, TEST_PORT, -1, "")) + ); + return serviceClient; + }; + ScanResultValue scanResultValue = new ScanResultValue( + null, + ImmutableList.of(), + ImmutableList.of( + ImmutableList.of("abc", "123"), + ImmutableList.of("ghi", "456"), + ImmutableList.of("xyz", "789") + ) + ); + + doAnswer( + invocation -> { +// ResponseContext responseContext = invocation.getArgument(1); + return new ByteArrayInputStream(new ObjectMapper().writeValueAsBytes(scanResultValue)); + } + ).when(serviceClient).request(any(), any()); + } + + @Test + public void fetchRowsFromDataServerWithScanQuery() + { + ScanQuery query = new ScanQuery( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")), + VirtualColumns.EMPTY, + ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST, + 0, + 0, + 3, + ScanQuery.Order.NONE, + null, + null, + Arrays.asList("dim1", "dim2"), + null, + null + ); + +// Sequence sequence = target.fetchRowsFromDataServer( +// query, +// ScanQueryFrameProcessor::mappingFunction, +// Closer.create() +// ); +// +// Assert.assertEquals(sequence.toList(), List.of( +// new Object[] {"abc", "123"}, +// new Object[] {"ghi", "456"}, +// new Object[] {"xyz", "789"} +// )); + } +} \ No newline at end of file diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index fb43ee839f8f..ab9b6cc5163f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -65,7 +65,7 @@ public class MSQLoadedSegmentTests extends MSQTestBase public static final Map REALTIME_QUERY_CTX = ImmutableMap.builder() .putAll(DEFAULT_MSQ_CONTEXT) - .put(MultiStageQueryContext.CTX_INCLUDE_REALTIME, true) + .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, MultiStageQueryContext.SegmentSource.REALTIME.name()) .build(); public static final DataSegment LOADED_SEGMENT_1 = DataSegment.builder() From 53bf3498600bcf129fa9170d194f58b7bc855b7b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 1 Oct 2023 22:00:38 +0530 Subject: [PATCH 20/37] Delete unwanted test --- .../LoadedSegmentDataProviderImplTest.java | 138 ------------------ 1 file changed, 138 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java deleted file mode 100644 index 716594058faa..000000000000 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImplTest.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.msq.exec; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.msq.test.MSQTestBase; -import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.query.spec.LegacySegmentSpec; -import org.apache.druid.rpc.ServiceClient; -import org.apache.druid.rpc.ServiceClientFactory; -import org.apache.druid.rpc.ServiceLocation; -import org.apache.druid.rpc.ServiceLocations; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -import java.io.ByteArrayInputStream; -import java.util.Arrays; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doAnswer; - -@RunWith(MockitoJUnitRunner.class) -public class LoadedSegmentDataProviderImplTest -{ - private final String TEST_HOST = "indexer"; - private final int TEST_PORT = 9092; - @Mock - ServiceClientFactory serviceClientFactory; - @Mock - ServiceClient serviceClient; - - @Mock - CoordinatorClient coordinatorClient; - final RichSegmentDescriptor testSegmentDescriptor = new RichSegmentDescriptor( - Intervals.ETERNITY, - Intervals.ETERNITY, - "v1", - 1, - ImmutableSet.of(new DruidServerMetadata("indexer", TEST_HOST + ":" + TEST_PORT, null, 1, ServerType.REALTIME, "tier", 2))); - - final LoadedSegmentDataProviderImpl target = new LoadedSegmentDataProviderImpl( - QueryRunnerTestHelper.DATA_SOURCE, new ChannelCounters(), serviceClientFactory, coordinatorClient, new ObjectMapper() - ); - - @Before - public void setUp() throws Exception - { - serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> { - Assert.assertEquals( - FutureUtils.getUnchecked(serviceLocator.locate(), true), - ServiceLocations.forLocation(new ServiceLocation(TEST_HOST, TEST_PORT, -1, "")) - ); - return serviceClient; - }; - ScanResultValue scanResultValue = new ScanResultValue( - null, - ImmutableList.of(), - ImmutableList.of( - ImmutableList.of("abc", "123"), - ImmutableList.of("ghi", "456"), - ImmutableList.of("xyz", "789") - ) - ); - - doAnswer( - invocation -> { -// ResponseContext responseContext = invocation.getArgument(1); - return new ByteArrayInputStream(new ObjectMapper().writeValueAsBytes(scanResultValue)); - } - ).when(serviceClient).request(any(), any()); - } - - @Test - public void fetchRowsFromDataServerWithScanQuery() - { - ScanQuery query = new ScanQuery( - new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), - new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")), - VirtualColumns.EMPTY, - ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST, - 0, - 0, - 3, - ScanQuery.Order.NONE, - null, - null, - Arrays.asList("dim1", "dim2"), - null, - null - ); - -// Sequence sequence = target.fetchRowsFromDataServer( -// query, -// ScanQueryFrameProcessor::mappingFunction, -// Closer.create() -// ); -// -// Assert.assertEquals(sequence.toList(), List.of( -// new Object[] {"abc", "123"}, -// new Object[] {"ghi", "456"}, -// new Object[] {"xyz", "789"} -// )); - } -} \ No newline at end of file From 45bdfa4e893ad43c8c661708831f21b59c34b734 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 2 Oct 2023 14:02:15 +0530 Subject: [PATCH 21/37] Add unit tests --- .../apache/druid/msq/exec/ControllerImpl.java | 5 +- .../exec/LoadedSegmentDataProviderImpl.java | 2 +- .../apache/druid/msq/exec/SegmentSource.java | 49 ++++++++ .../msq/util/MultiStageQueryContext.java | 26 +--- .../druid/msq/exec/MSQLoadedSegmentTests.java | 2 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 15 ++- .../druid/discovery/DataServerClient.java | 2 +- .../druid/rpc/FixedSetServiceLocator.java | 20 +-- .../druid/discovery/DataServerClientTest.java | 115 ++++++++++++++++++ .../druid/rpc/FixedSetServiceLocatorTest.java | 63 ++++++++++ 10 files changed, 258 insertions(+), 41 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java create mode 100644 server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java create mode 100644 server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 397cf5d1e586..3690e04f81f4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1163,14 +1163,13 @@ private QueryKit makeQueryControllerToolKit() private DataSegmentTimelineView makeDataSegmentTimelineView() { - final MultiStageQueryContext.SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources( + final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources( task.getQuerySpec() .getQuery() .context() ); - final boolean includeRealtime - = MultiStageQueryContext.SegmentSource.shouldQueryRealtimeServers(includeSegmentSource); + final boolean includeRealtime = SegmentSource.shouldQueryRealtimeServers(includeSegmentSource); return (dataSource, intervals) -> { final Iterable realtimeAndHistoricalSegments; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 85faa1680a71..61b1fe0e0a62 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -93,7 +93,7 @@ public Pair> fetchR Set servers = segmentDescriptor.getServers(); final DataServerClient dataServerClient = new DataServerClient<>( serviceClientFactory, - new FixedSetServiceLocator(servers), + FixedSetServiceLocator.forDruidServerMetadata(servers), objectMapper ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java new file mode 100644 index 000000000000..5e61ce9c6ac7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.server.coordination.ServerType; + +import java.util.Set; + +public enum SegmentSource +{ + + NONE(ImmutableSet.of()), + REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR)); + + private final Set usedServerTypes; + + SegmentSource(Set usedServerTypes) + { + this.usedServerTypes = usedServerTypes; + } + + public Set getUsedServerTypes() + { + return usedServerTypes; + } + + public static boolean shouldQueryRealtimeServers(SegmentSource segmentSource) + { + return REALTIME.equals(segmentSource); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index d7d08e3648fc..e49e189e7759 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -23,25 +23,23 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableSet; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Limits; +import org.apache.druid.msq.exec.SegmentSource; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.server.coordination.ServerType; import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -316,26 +314,4 @@ static IndexSpec decodeIndexSpec(@Nullable final Object indexSpecObject, final O throw QueryContexts.badValueException(CTX_INDEX_SPEC, "an indexSpec", indexSpecObject); } } - - public enum SegmentSource { - - NONE(ImmutableSet.of()), - REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR)); - - private final Set usedServerTypes; - - SegmentSource(Set usedServerTypes) - { - this.usedServerTypes = usedServerTypes; - } - - public Set getUsedServerTypes() - { - return usedServerTypes; - } - - public static boolean shouldQueryRealtimeServers(SegmentSource segmentSource) { - return REALTIME.equals(segmentSource); - } - } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index ab9b6cc5163f..20752ef16e3c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -65,7 +65,7 @@ public class MSQLoadedSegmentTests extends MSQTestBase public static final Map REALTIME_QUERY_CTX = ImmutableMap.builder() .putAll(DEFAULT_MSQ_CONTEXT) - .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, MultiStageQueryContext.SegmentSource.REALTIME.name()) + .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name()) .build(); public static final DataSegment LOADED_SEGMENT_1 = DataSegment.builder() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 2cefd0a33eb0..13f7d1e8862a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; @@ -97,10 +98,11 @@ import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; +import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; -import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS; /** * Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests @@ -191,8 +193,15 @@ public String getFormatString() private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() { LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); - doThrow(new AssertionError("Test does not support loaded segment query")) - .when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); + LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); + try { + doThrow(new AssertionError("Test does not support loaded segment query")) + .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any(), any()); + doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); + } + catch (IOException e) { + throw new RuntimeException(e); + } return mockFactory; } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 6918d2291cab..67cdc6a84a33 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -64,7 +64,7 @@ public DataServerClient( ObjectMapper objectMapper ) { - serviceClient = serviceClientFactory.makeClient( + this.serviceClient = serviceClientFactory.makeClient( NodeRole.INDEXER.getJsonName(), fixedSetServiceLocator, StandardRetryPolicy.noRetries() diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java index 33d4cdd89d56..a5149ceb7346 100644 --- a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -35,15 +35,21 @@ public class FixedSetServiceLocator implements ServiceLocator { private ServiceLocations serviceLocations; - public FixedSetServiceLocator(Set servers) + private FixedSetServiceLocator(ServiceLocations serviceLocations) { - if (servers == null) { - serviceLocations = ServiceLocations.closed(); + this.serviceLocations = serviceLocations; + } + + public static FixedSetServiceLocator forDruidServerMetadata(Set serverMetadataSet) + { + if (serverMetadataSet == null || serverMetadataSet.isEmpty()) { + return new FixedSetServiceLocator(ServiceLocations.closed()); } else { - Set serviceLocationSet = servers.stream() - .map(ServiceLocation::fromDruidServerMetadata) - .collect(Collectors.toSet()); - serviceLocations = ServiceLocations.forLocations(serviceLocationSet); + Set serviceLocationSet = serverMetadataSet.stream() + .map(ServiceLocation::fromDruidServerMetadata) + .collect(Collectors.toSet()); + + return new FixedSetServiceLocator(ServiceLocations.forLocations(serviceLocationSet)); } } diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java new file mode 100644 index 000000000000..39758f8530fd --- /dev/null +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.discovery; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.context.DefaultResponseContext; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.MockServiceClient; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClientFactory; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import java.util.Collections; + +import static org.apache.druid.query.Druids.newScanQueryBuilder; +import static org.mockito.Mockito.mock; + +public class DataServerClientTest +{ + MockServiceClient serviceClient; + ServiceClientFactory serviceClientFactory; + ObjectMapper jsonMapper; + ScanQuery query; + DataServerClient target; + + @Before + public void setUp() throws Exception + { + jsonMapper = DruidServiceTestUtils.newJsonMapper(); + serviceClient = new MockServiceClient(); + serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> serviceClient; + + query = newScanQueryBuilder() + .dataSource("dataSource1") + .intervals( + new MultipleSpecificSegmentSpec( + ImmutableList.of( + new SegmentDescriptor(Intervals.of("2003/2004"), "v0", 1) + ) + ) + ) + .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build(); + + target = new DataServerClient( + serviceClientFactory, + mock(FixedSetServiceLocator.class), + jsonMapper + ); + } + + @Test + public void testFetchSegmentFromDataServer() throws JsonProcessingException + { + ScanResultValue scanResultValue = new ScanResultValue( + null, + ImmutableList.of("id", "name"), + ImmutableList.of( + ImmutableList.of(1, "abc"), + ImmutableList.of(5, "efg") + )); + + RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/druid/v2/") + .jsonContent(jsonMapper, query); + serviceClient.expectAndRespond( + requestBuilder, + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(Collections.singletonList(scanResultValue)) + ); + + ResponseContext responseContext = new DefaultResponseContext(); + Sequence result = target.run( + query, + responseContext, + jsonMapper.getTypeFactory().constructType(ScanResultValue.class) + ); + + Assert.assertEquals(ImmutableList.of(scanResultValue), result.toList()); + } +} diff --git a/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java b/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java new file mode 100644 index 000000000000..e366f6030346 --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.rpc; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.ExecutionException; + +public class FixedSetServiceLocatorTest +{ + public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( + "TestDataServer", + "hostName:9092", + null, + 2, + ServerType.REALTIME, + "tier1", + 2 + ); + + @Test + public void testLocateNullShouldBeClosed() throws ExecutionException, InterruptedException + { + FixedSetServiceLocator serviceLocator + = FixedSetServiceLocator.forDruidServerMetadata(null); + + Assert.assertTrue(serviceLocator.locate().get().isClosed()); + } + + + @Test + public void testLocateSingleServer() throws ExecutionException, InterruptedException + { + FixedSetServiceLocator serviceLocator + = FixedSetServiceLocator.forDruidServerMetadata(ImmutableSet.of(DATA_SERVER_1)); + + Assert.assertEquals( + ServiceLocations.forLocation(ServiceLocation.fromDruidServerMetadata(DATA_SERVER_1)), + serviceLocator.locate().get() + ); + } +} From 85c4923949e69da489cf75f53761507609943816 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 2 Oct 2023 22:59:52 +0530 Subject: [PATCH 22/37] Add unit tests --- .../exec/LoadedSegmentDataProviderImpl.java | 26 +++++++++++-------- .../input/table/RichSegmentDescriptor.java | 15 ++++------- .../input/table/SegmentWithDescriptor.java | 7 +++-- .../druid/msq/exec/MSQLoadedSegmentTests.java | 1 - .../table/RichSegmentDescriptorTest.java | 19 +++++++++++--- .../druid/discovery/DataServerClient.java | 6 ++--- 6 files changed, 44 insertions(+), 30 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 61b1fe0e0a62..296be8ea9f73 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; @@ -76,6 +77,12 @@ public LoadedSegmentDataProviderImpl( this.objectMapper = objectMapper; } + @VisibleForTesting + DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) + { + return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper); + } + @Override public Pair> fetchRowsFromDataServer( Query query, @@ -91,11 +98,7 @@ public Pair> fetchR ); Set servers = segmentDescriptor.getServers(); - final DataServerClient dataServerClient = new DataServerClient<>( - serviceClientFactory, - FixedSetServiceLocator.forDruidServerMetadata(servers), - objectMapper - ); + DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); @@ -108,16 +111,17 @@ public Pair> fetchR statusSequencePair = RetryUtils.retry( () -> { Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); - Yielder yielder = Yielders.each(mappingFunction.apply(sequence) - .map(row -> { - channelCounters.incrementRowCount(); - return row; - })); - closer.register(yielder); final List missingSegments = getMissingSegments(responseContext); if (missingSegments.isEmpty()) { log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); // Segment was found + Yielder yielder = closer.register( + Yielders.each(mappingFunction.apply(sequence) + .map(row -> { + channelCounters.incrementRowCount(); + return row; + })) + ); return Pair.of(DataServerQueryStatus.SUCCESS, yielder); } else { Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index b99b50abf791..d19ee5f2155c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -46,7 +46,6 @@ public class RichSegmentDescriptor extends SegmentDescriptor { @Nullable private final Interval fullInterval; - @Nullable private final Set servers; public RichSegmentDescriptor( @@ -65,7 +64,7 @@ public RichSegmentDescriptor( public RichSegmentDescriptor( SegmentDescriptor segmentDescriptor, @Nullable Interval fullInterval, - @Nullable Set servers + Set servers ) { super(segmentDescriptor.getInterval(), segmentDescriptor.getVersion(), segmentDescriptor.getPartitionNumber()); @@ -92,8 +91,6 @@ static RichSegmentDescriptor fromJson( } @JsonProperty("servers") - @JsonInclude(JsonInclude.Include.NON_NULL) - @Nullable public Set getServers() { return servers; @@ -125,23 +122,21 @@ public boolean equals(Object o) return false; } RichSegmentDescriptor that = (RichSegmentDescriptor) o; - return Objects.equals(fullInterval, that.fullInterval); + return Objects.equals(fullInterval, that.fullInterval) && Objects.equals(servers, that.servers); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), fullInterval); + return Objects.hash(super.hashCode(), fullInterval, servers); } @Override public String toString() { return "RichSegmentDescriptor{" + - "fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) + - ", interval=" + getInterval() + - ", version='" + getVersion() + '\'' + - ", partitionNumber=" + getPartitionNumber() + + "fullInterval=" + fullInterval + + ", servers=" + servers + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 98822051b7e3..191b2d87eed8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -112,12 +112,15 @@ public boolean equals(Object o) return false; } SegmentWithDescriptor that = (SegmentWithDescriptor) o; - return Objects.equals(segmentSupplier, that.segmentSupplier) && Objects.equals(descriptor, that.descriptor); + return Objects.equals(segmentSupplier, that.segmentSupplier) && Objects.equals( + loadedSegmentDataProvider, + that.loadedSegmentDataProvider + ) && Objects.equals(descriptor, that.descriptor); } @Override public int hashCode() { - return Objects.hash(segmentSupplier, descriptor); + return Objects.hash(segmentSupplier, loadedSegmentDataProvider, descriptor); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index 20752ef16e3c..ad0d087ad9b5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -270,7 +270,6 @@ public void testDataServerQueryFailedShouldFail() throws IOException ) .when(loadedSegmentDataProvider) .fetchRowsFromDataServer(any(), any(), any(), any(), any()); - doReturn(Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, null)); testSelectQuery() .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java index d1826d6cfd01..935b464e0386 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java @@ -20,15 +20,28 @@ package org.apache.druid.msq.input.table; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.TestHelper; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.junit.Assert; import org.junit.Test; public class RichSegmentDescriptorTest { + private static final DruidServerMetadata DRUID_SERVER_1 = new DruidServerMetadata( + "name1", + "host1", + null, + 100L, + ServerType.REALTIME, + "tier1", + 0 + ); + @Test public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception { @@ -38,7 +51,7 @@ public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception Intervals.of("2000/2001"), "2", 3, - null + ImmutableSet.of(DRUID_SERVER_1) ); Assert.assertEquals( @@ -56,7 +69,7 @@ public void testSerdeWithFullIntervalSameAsInterval() throws Exception Intervals.of("2000/2001"), "2", 3, - null + ImmutableSet.of(DRUID_SERVER_1) ); Assert.assertEquals( @@ -74,7 +87,7 @@ public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exc Intervals.of("2000/2001"), "2", 3, - null + ImmutableSet.of(DRUID_SERVER_1) ); Assert.assertEquals( diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 67cdc6a84a33..0ac555efe9dc 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -51,7 +51,7 @@ /** * Client to query data servers given a query. */ -public class DataServerClient +public class DataServerClient { private static final Logger log = new Logger(DataServerClient.class); private final ServiceClient serviceClient; @@ -73,7 +73,7 @@ public DataServerClient( this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } - public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType) + public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType) { final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); @@ -138,7 +138,7 @@ public void cleanup(JsonParserIterator iterFromMake) ); } - private void cancelQuery(Query query, String cancelPath) + private void cancelQuery(Query query, String cancelPath) { Runnable cancelRunnable = () -> { Future cancelFuture = serviceClient.asyncRequest( From c3222cced78cc2d18f0de5b7580017575ce3a229 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 3 Oct 2023 10:39:42 +0530 Subject: [PATCH 23/37] Fix tests --- .../druid/msq/exec/LoadedSegmentDataProvider.java | 3 +-- .../msq/exec/LoadedSegmentDataProviderImpl.java | 2 +- .../msq/input/table/SegmentsInputSliceTest.java | 15 ++++++++++++++- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index c904f870a18e..d0522c8ac0ae 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -42,7 +42,6 @@ Pair> fetchRowsFrom enum DataServerQueryStatus { SUCCESS, - HANDOFF, - FAILED + HANDOFF } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java index 296be8ea9f73..40d0748fa63d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java @@ -144,7 +144,7 @@ public Pair> fetchR return statusSequencePair; } catch (Exception e) { - log.error("Exception while fetching rows from dataservers."); + log.error(e, "Exception while fetching rows for query[%s] from dataservers[%s].", query, servers); throw new IOE(e, "Exception while fetching rows from dataservers."); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index 346f3e15ebf2..55bb424512d8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -21,11 +21,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.segment.TestHelper; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.junit.Assert; import org.junit.Test; @@ -45,7 +48,17 @@ public void testSerde() throws Exception Intervals.of("2000/P1M"), "1", 0, - null + ImmutableSet.of( + new DruidServerMetadata( + "name1", + "host1", + null, + 100L, + ServerType.REALTIME, + "tier1", + 0 + ) + ) ) ) ); From 5587fd58abbe8a5949c43273b2ad3709b2938c81 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 3 Oct 2023 12:12:15 +0530 Subject: [PATCH 24/37] Add documentation --- docs/multi-stage-query/reference.md | 1 + .../msq/exec/LoadedSegmentDataProvider.java | 24 +++++++++++++++++++ .../input/table/RichSegmentDescriptor.java | 6 ++--- .../druid/msq/exec/MSQLoadedSegmentTests.java | 2 +- .../druid/discovery/DataServerClientTest.java | 2 +- .../sql/calcite/BaseCalciteQueryTest.java | 2 +- 6 files changed, 31 insertions(+), 6 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 6236b6545258..2137cb61a2d2 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -246,6 +246,7 @@ The following table lists the context parameters for the MSQ task engine: | `durableShuffleStorage` | SELECT, INSERT, REPLACE

Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error.

| `false` | | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | | `selectDestination` | SELECT

Controls where the final result of the select query is written.
Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results
Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` | +| `includeSegmentSource` | SELECT, INSERT, REPLACE

Controls the sources, which will be queried for results, besides the segments in deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only published used segments will be considered. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` | ## Joins diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index d0522c8ac0ae..661a13ad53dd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -29,8 +29,22 @@ import java.io.IOException; import java.util.function.Function; +/** + * Class responsible for querying dataservers and retriving results for a given query. Also queries the coordinator + * to check if a segment has been handed off. + */ public interface LoadedSegmentDataProvider { + /** + * Queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver indicates + * that the segment was not found, checks with the coordinator to see if the segment was handed off. + * - If the segment was handed off, returns with a {@link DataServerQueryStatus#HANDOFF} status. + * - If the segment was not handed off, retries with the known list of servers and throws an exception if the retry + * count is exceeded. + * + * @param result return type for the query from the data server + * @param type of the result rows after parsing from QueryType object + */ Pair> fetchRowsFromDataServer( Query query, RichSegmentDescriptor segmentDescriptor, @@ -39,9 +53,19 @@ Pair> fetchRowsFrom Closer closer ) throws IOException; + /** + * Represents the status of fetching a segment from a data server + */ enum DataServerQueryStatus { + /** + * Segment was found on the data server and fetched successfully. + */ SUCCESS, + /** + * Segment was not found on the realtime server as it has been handed off to a historical. Only returned while + * querying a realtime server. + */ HANDOFF } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index d19ee5f2155c..ed8c8f7ff826 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -33,14 +33,14 @@ import java.util.Set; /** - * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment. - * (SegmentDescriptor only provides the clipped interval.) + * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment + * (SegmentDescriptor only provides the clipped interval.), as well as the metadata of the servers it is loaded on. *
* To keep the serialized form lightweight, the full interval is only serialized if it is different from the * clipped interval. *
* It is possible to deserialize this class as {@link SegmentDescriptor}. However, going the other direction is - * not a good idea, because the {@link #fullInterval} will not end up being set correctly. + * not a good idea, because the {@link #fullInterval} and {@link #servers} will not end up being set correctly. */ public class RichSegmentDescriptor extends SegmentDescriptor { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index ad0d087ad9b5..73509e37f78c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -87,7 +87,7 @@ public class MSQLoadedSegmentTests extends MSQTestBase ); @Before - public void setUp() throws Exception + public void setUp() { loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); } diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index 39758f8530fd..b7569982056b 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -57,7 +57,7 @@ public class DataServerClientTest DataServerClient target; @Before - public void setUp() throws Exception + public void setUp() { jsonMapper = DruidServiceTestUtils.newJsonMapper(); serviceClient = new MockServiceClient(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 84fd4217c75a..04c6f686fa5f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1058,7 +1058,7 @@ public void assertResultsEquals(String sql, List expectedResults, List { for (int i = 0; i < results.size(); i++) { Assert.assertArrayEquals( - StringUtils.format("result #%d: %s", i + 1, sql), + StringUtils.format("Expected %s, actual %s: result #%d: %s", expectedResults, results, i + 1, sql), expectedResults.get(i), results.get(i) ); From cc3db10e4947ddc5375769100b581cc020dc333d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 3 Oct 2023 13:50:12 +0530 Subject: [PATCH 25/37] Fix tests --- .../druid/msq/exec/MSQLoadedSegmentTests.java | 26 +++++++++---------- .../sql/calcite/BaseCalciteQueryTest.java | 2 +- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index 73509e37f78c..cdc8fbdf714f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -96,8 +96,8 @@ public void setUp() public void testSelectWithLoadedSegmentsOnFoo() throws IOException { RowSignature resultSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) .build(); doReturn( @@ -106,8 +106,8 @@ public void testSelectWithLoadedSegmentsOnFoo() throws IOException Yielders.each( Sequences.simple( ImmutableList.of( - new Object[]{"5", "qwe"}, - new Object[]{"123", "tyu"} + new Object[]{1L, "qwe"}, + new Object[]{1L, "tyu"} ) ) ) @@ -117,14 +117,14 @@ public void testSelectWithLoadedSegmentsOnFoo() throws IOException .fetchRowsFromDataServer(any(), any(), any(), any(), any()); testSelectQuery() - .setSql("select dim1, dim2 from foo") + .setSql("select cnt, dim1 from foo") .setExpectedMSQSpec( MSQSpec.builder() .query( newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .columns("dim1", "dim2") + .columns("cnt", "dim1") .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) .build() ) @@ -136,14 +136,14 @@ public void testSelectWithLoadedSegmentsOnFoo() throws IOException .setQueryContext(REALTIME_QUERY_CTX) .setExpectedRowSignature(resultSignature) .setExpectedResultRows(ImmutableList.of( - new Object[]{"", "a"}, - new Object[]{"5", "qwe"}, - new Object[]{"10.1", null}, - new Object[]{"123", "tyu"}, - new Object[]{"2", ""}, - new Object[]{"1", "a"}, - new Object[]{"def", "abc"}, - new Object[]{"abc", null} + new Object[]{1L, ""}, + new Object[]{1L, "qwe"}, + new Object[]{1L, "10.1"}, + new Object[]{1L, "tyu"}, + new Object[]{1L, "2"}, + new Object[]{1L, "1"}, + new Object[]{1L, "def"}, + new Object[]{1L, "abc"} )) .verifyResults(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 04c6f686fa5f..84fd4217c75a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1058,7 +1058,7 @@ public void assertResultsEquals(String sql, List expectedResults, List { for (int i = 0; i < results.size(); i++) { Assert.assertArrayEquals( - StringUtils.format("Expected %s, actual %s: result #%d: %s", expectedResults, results, i + 1, sql), + StringUtils.format("result #%d: %s", i + 1, sql), expectedResults.get(i), results.get(i) ); From ed5b461c9c1677f2aee76230cebfa09d720de3cd Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 3 Oct 2023 19:53:47 +0530 Subject: [PATCH 26/37] Add tests and remove interface --- .../msq/exec/LoadedSegmentDataProvider.java | 144 ++++++++++++- .../LoadedSegmentDataProviderFactory.java | 2 +- .../exec/LoadedSegmentDataProviderImpl.java | 175 ---------------- .../exec/LoadedSegmentDataProviderTest.java | 192 ++++++++++++++++++ 4 files changed, 332 insertions(+), 181 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 661a13ad53dd..5d18f72bb476 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -19,22 +19,74 @@ package org.apache.druid.msq.exec; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.query.Queries; import org.apache.druid.query.Query; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.context.DefaultResponseContext; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.server.coordination.DruidServerMetadata; import java.io.IOException; +import java.util.List; +import java.util.Set; import java.util.function.Function; /** * Class responsible for querying dataservers and retriving results for a given query. Also queries the coordinator * to check if a segment has been handed off. */ -public interface LoadedSegmentDataProvider +public class LoadedSegmentDataProvider { + private static final Logger log = new Logger(LoadedSegmentDataProvider.class); + private static final int DEFAULT_NUM_TRIES = 5; + private final String dataSource; + private final ChannelCounters channelCounters; + private final ServiceClientFactory serviceClientFactory; + private final CoordinatorClient coordinatorClient; + private final ObjectMapper objectMapper; + + public LoadedSegmentDataProvider( + String dataSource, + ChannelCounters channelCounters, + ServiceClientFactory serviceClientFactory, + CoordinatorClient coordinatorClient, + ObjectMapper objectMapper + ) + { + this.dataSource = dataSource; + this.channelCounters = channelCounters; + this.serviceClientFactory = serviceClientFactory; + this.coordinatorClient = coordinatorClient; + this.objectMapper = objectMapper; + } + + @VisibleForTesting + DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) + { + return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper); + } + /** * Queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver indicates * that the segment was not found, checks with the coordinator to see if the segment was handed off. @@ -45,18 +97,100 @@ public interface LoadedSegmentDataProvider * @param result return type for the query from the data server * @param type of the result rows after parsing from QueryType object */ - Pair> fetchRowsFromDataServer( + public Pair> fetchRowsFromDataServer( Query query, RichSegmentDescriptor segmentDescriptor, Function, Sequence> mappingFunction, - Class queryResultType, + Class resultClass, Closer closer - ) throws IOException; + ) throws IOException + { + final Query preparedQuery = Queries.withSpecificSegments( + query.withDataSource(new TableDataSource(dataSource)), + ImmutableList.of(segmentDescriptor) + ); + + Set servers = segmentDescriptor.getServers(); + DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); + + final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); + final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); + + log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); + final ResponseContext responseContext = new DefaultResponseContext(); + + Pair> statusSequencePair; + try { + statusSequencePair = RetryUtils.retry( + () -> { + Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); + final List missingSegments = getMissingSegments(responseContext); + // Only one segment is fetched, so this should be empty if it was fetched + if (missingSegments.isEmpty()) { + log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); + // Segment was found + Yielder yielder = closer.register( + Yielders.each(mappingFunction.apply(sequence) + .map(row -> { + channelCounters.incrementRowCount(); + return row; + })) + ); + return Pair.of(DataServerQueryStatus.SUCCESS, yielder); + } else { + Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); + if (Boolean.TRUE.equals(wasHandedOff)) { + log.debug("Segment[%s] was handed off.", segmentDescriptor); + return Pair.of(DataServerQueryStatus.HANDOFF, null); + } else { + log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor); + throw new ISE( + "Segment[%s] could not be found on data server, but segment was not handed off.", + segmentDescriptor + ); + } + } + }, + input -> true, + numRetriesOnMissingSegments + ); + + return statusSequencePair; + } + catch (Exception e) { + log.error(e, "Exception while fetching rows for query[%s] from dataservers[%s].", query, servers); + throw new IOE(e, "Exception while fetching rows from dataservers."); + } + } + + private static List getMissingSegments(final ResponseContext responseContext) + { + List missingSegments = responseContext.getMissingSegments(); + if (missingSegments == null) { + return ImmutableList.of(); + } + return missingSegments; + } + + private static boolean checkSegmentHandoff( + CoordinatorClient coordinatorClient, + String dataSource, + SegmentDescriptor segmentDescriptor + ) throws Exception + { + Boolean wasHandedOff = RetryUtils.retry( + () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), + input -> true, + RetryUtils.DEFAULT_MAX_TRIES + ); + + return Boolean.TRUE.equals(wasHandedOff); + } /** * Represents the status of fetching a segment from a data server */ - enum DataServerQueryStatus + public enum DataServerQueryStatus { /** * Segment was found on the data server and fetched successfully. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 2ca2d636bf45..63e5f8bd0ac0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -47,7 +47,7 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( ChannelCounters channelCounters ) { - return new LoadedSegmentDataProviderImpl( + return new LoadedSegmentDataProvider( dataSource, channelCounters, serviceClientFactory, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java deleted file mode 100644 index 40d0748fa63d..000000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderImpl.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.msq.exec; - -import com.fasterxml.jackson.databind.JavaType; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.discovery.DataServerClient; -import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.query.Queries; -import org.apache.druid.query.Query; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.context.DefaultResponseContext; -import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.rpc.FixedSetServiceLocator; -import org.apache.druid.rpc.ServiceClientFactory; -import org.apache.druid.server.coordination.DruidServerMetadata; - -import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.function.Function; - -public class LoadedSegmentDataProviderImpl implements LoadedSegmentDataProvider -{ - private static final Logger log = new Logger(LoadedSegmentDataProviderImpl.class); - private static final int DEFAULT_NUM_TRIES = 5; - private final String dataSource; - private final ChannelCounters channelCounters; - private final ServiceClientFactory serviceClientFactory; - private final CoordinatorClient coordinatorClient; - private final ObjectMapper objectMapper; - - public LoadedSegmentDataProviderImpl( - String dataSource, - ChannelCounters channelCounters, - ServiceClientFactory serviceClientFactory, - CoordinatorClient coordinatorClient, - ObjectMapper objectMapper - ) - { - this.dataSource = dataSource; - this.channelCounters = channelCounters; - this.serviceClientFactory = serviceClientFactory; - this.coordinatorClient = coordinatorClient; - this.objectMapper = objectMapper; - } - - @VisibleForTesting - DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) - { - return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper); - } - - @Override - public Pair> fetchRowsFromDataServer( - Query query, - RichSegmentDescriptor segmentDescriptor, - Function, Sequence> mappingFunction, - Class resultClass, - Closer closer - ) throws IOException - { - final Query preparedQuery = Queries.withSpecificSegments( - query.withDataSource(new TableDataSource(dataSource)), - ImmutableList.of(segmentDescriptor) - ); - - Set servers = segmentDescriptor.getServers(); - DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); - - final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); - final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); - - log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); - final ResponseContext responseContext = new DefaultResponseContext(); - - Pair> statusSequencePair; - try { - statusSequencePair = RetryUtils.retry( - () -> { - Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); - final List missingSegments = getMissingSegments(responseContext); - if (missingSegments.isEmpty()) { - log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); - // Segment was found - Yielder yielder = closer.register( - Yielders.each(mappingFunction.apply(sequence) - .map(row -> { - channelCounters.incrementRowCount(); - return row; - })) - ); - return Pair.of(DataServerQueryStatus.SUCCESS, yielder); - } else { - Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); - if (Boolean.TRUE.equals(wasHandedOff)) { - log.debug("Segment[%s] was handed off.", segmentDescriptor); - return Pair.of(DataServerQueryStatus.HANDOFF, null); - } else { - log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor); - throw new ISE( - "Segment[%s] could not be found on data server, but segment was not handed off.", - segmentDescriptor - ); - } - } - }, - input -> true, - numRetriesOnMissingSegments - ); - - return statusSequencePair; - } - catch (Exception e) { - log.error(e, "Exception while fetching rows for query[%s] from dataservers[%s].", query, servers); - throw new IOE(e, "Exception while fetching rows from dataservers."); - } - } - - private static List getMissingSegments(final ResponseContext responseContext) - { - List missingSegments = responseContext.getMissingSegments(); - if (missingSegments == null) { - return ImmutableList.of(); - } - return missingSegments; - } - - private static boolean checkSegmentHandoff( - CoordinatorClient coordinatorClient, - String dataSource, - SegmentDescriptor segmentDescriptor - ) throws Exception - { - Boolean wasHandedOff = RetryUtils.retry( - () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), - input -> true, - RetryUtils.DEFAULT_MAX_TRIES - ); - - return Boolean.TRUE.equals(wasHandedOff); - } -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java new file mode 100644 index 000000000000..430321fc7f6b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Futures; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.discovery.DruidServiceTestUtils; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.querykit.InputNumberDataSource; +import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import java.io.IOException; +import java.util.List; + +import static org.apache.druid.query.Druids.newScanQueryBuilder; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +@RunWith(MockitoJUnitRunner.class) +public class LoadedSegmentDataProviderTest +{ + private static final String DATASOURCE1 = "dataSource1"; + private static final DruidServerMetadata DRUID_SERVER_1 = new DruidServerMetadata( + "name1", + "host1:5050", + null, + 100L, + ServerType.REALTIME, + "tier1", + 0 + ); + private static final RichSegmentDescriptor SEGMENT_1 = new RichSegmentDescriptor( + Intervals.of("2003/2004"), + Intervals.of("2003/2004"), + "v1", + 1, + ImmutableSet.of(DRUID_SERVER_1) + ); + private DataServerClient dataServerClient; + private CoordinatorClient coordinatorClient; + private ScanResultValue scanResultValue; + private ScanQuery query; + private LoadedSegmentDataProvider target; + + @Before + public void setUp() throws Exception + { + dataServerClient = mock(DataServerClient.class); + coordinatorClient = mock(CoordinatorClient.class); + scanResultValue = new ScanResultValue( + null, + ImmutableList.of(), + ImmutableList.of( + ImmutableList.of("abc", "123"), + ImmutableList.of("ghi", "456"), + ImmutableList.of("xyz", "789") + ) + ); + query = newScanQueryBuilder() + .dataSource(new InputNumberDataSource(1)) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003/2004")))) + .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)) + .build(); + target = spy( + new LoadedSegmentDataProvider( + DATASOURCE1, + new ChannelCounters(), + mock(ServiceClientFactory.class), + coordinatorClient, + DruidServiceTestUtils.newJsonMapper() + ) + ); + doReturn(dataServerClient).when(target).makeDataServerClient(any()); + } + + @Test + public void testFetchRowsFromServer() throws IOException + { + doReturn(Sequences.simple(ImmutableList.of(scanResultValue))) + .when(dataServerClient).run(any(), any(), any()); + + Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( + query, + SEGMENT_1, + ScanQueryFrameProcessor::mappingFunction, + ScanResultValue.class, + Closer.create() + ); + + Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, dataServerQueryStatusYielderPair.lhs); + List> events = (List>) scanResultValue.getEvents(); + Yielder yielder = dataServerQueryStatusYielderPair.rhs; + events.forEach( + event -> { + Assert.assertArrayEquals(event.toArray(), yielder.get()); + yielder.next(null); + } + ); + } + + @Test + public void testHandoff() throws IOException + { + doAnswer(invocation -> { + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); + return Sequences.empty(); + } + ).when(dataServerClient).run(any(), any(), any()); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + + Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( + query, + SEGMENT_1, + ScanQueryFrameProcessor::mappingFunction, + ScanResultValue.class, + Closer.create() + ); + + Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs); + Assert.assertNull(dataServerQueryStatusYielderPair.rhs); + } + + + @Test + public void testQueryFail() + { + doAnswer(invocation -> { + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); + return Sequences.empty(); + } + ).when(dataServerClient).run(any(), any(), any()); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + + Assert.assertThrows(IOE.class, () -> + target.fetchRowsFromDataServer( + query, + SEGMENT_1, + ScanQueryFrameProcessor::mappingFunction, + ScanResultValue.class, + Closer.create() + ) + ); + } +} From 7fedc60f94534217059618be585e99a9382fc655 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 4 Oct 2023 08:50:57 +0530 Subject: [PATCH 27/37] Add check for case where realtime servers cannot be contacted --- .../msq/exec/LoadedSegmentDataProvider.java | 40 ++++++++++++++----- 1 file changed, 30 insertions(+), 10 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 5d18f72bb476..f4c605393e48 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -22,12 +22,12 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; @@ -39,11 +39,13 @@ import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.rpc.FixedSetServiceLocator; +import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -144,7 +146,7 @@ public Pair> fetchR return Pair.of(DataServerQueryStatus.HANDOFF, null); } else { log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor); - throw new ISE( + throw new IOE( "Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor ); @@ -157,9 +159,21 @@ public Pair> fetchR return statusSequencePair; } + catch (QueryInterruptedException e) { + if (e.getCause() instanceof RpcException) { + // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), + // we would also be unable to fetch the segment. Check if the segment was handed off, just in case. + boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); + if (wasHandedOff) { + log.debug("Segment[%s] was handed off.", segmentDescriptor); + return Pair.of(DataServerQueryStatus.HANDOFF, null); + } + } + throw new IOE(e, "Exception while fetching rows for query from dataservers[%s]", servers); + } catch (Exception e) { - log.error(e, "Exception while fetching rows for query[%s] from dataservers[%s].", query, servers); - throw new IOE(e, "Exception while fetching rows from dataservers."); + Throwables.propagateIfPossible(e, IOE.class); + throw new IOE(e, "Exception while fetching rows for query from dataservers[%s]", servers); } } @@ -176,13 +190,19 @@ private static boolean checkSegmentHandoff( CoordinatorClient coordinatorClient, String dataSource, SegmentDescriptor segmentDescriptor - ) throws Exception + ) throws IOE { - Boolean wasHandedOff = RetryUtils.retry( - () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), - input -> true, - RetryUtils.DEFAULT_MAX_TRIES - ); + Boolean wasHandedOff; + try { + wasHandedOff = RetryUtils.retry( + () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), + input -> true, + DEFAULT_NUM_TRIES + ); + } + catch (Exception e) { + throw new IOE(e, "Could not contact coordinator"); + } return Boolean.TRUE.equals(wasHandedOff); } From e4792f9bbae45dd1f602f8cc1bc4becc37439a69 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 4 Oct 2023 10:32:40 +0530 Subject: [PATCH 28/37] Cancel query on interrupt --- .../msq/exec/LoadedSegmentDataProvider.java | 4 ++-- .../exec/LoadedSegmentDataProviderTest.java | 21 ++++++++----------- .../druid/discovery/DataServerClient.java | 4 +++- .../druid/discovery/DataServerClientTest.java | 4 +++- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index f4c605393e48..760a3f7456fd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -125,7 +125,7 @@ public Pair> fetchR try { statusSequencePair = RetryUtils.retry( () -> { - Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType); + Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer); final List missingSegments = getMissingSegments(responseContext); // Only one segment is fetched, so this should be empty if it was fetched if (missingSegments.isEmpty()) { @@ -153,7 +153,7 @@ public Pair> fetchR } } }, - input -> true, + throwable -> !(throwable instanceof QueryInterruptedException && throwable.getCause() instanceof InterruptedException), numRetriesOnMissingSegments ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java index 430321fc7f6b..8412cb5d7a93 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java @@ -122,8 +122,7 @@ public void setUp() throws Exception @Test public void testFetchRowsFromServer() throws IOException { - doReturn(Sequences.simple(ImmutableList.of(scanResultValue))) - .when(dataServerClient).run(any(), any(), any()); + doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient).run(any(), any(), any(), any()); Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( query, @@ -148,11 +147,10 @@ public void testFetchRowsFromServer() throws IOException public void testHandoff() throws IOException { doAnswer(invocation -> { - ResponseContext responseContext = invocation.getArgument(1); - responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); - return Sequences.empty(); - } - ).when(dataServerClient).run(any(), any(), any()); + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); + return Sequences.empty(); + }).when(dataServerClient).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( @@ -172,11 +170,10 @@ public void testHandoff() throws IOException public void testQueryFail() { doAnswer(invocation -> { - ResponseContext responseContext = invocation.getArgument(1); - responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); - return Sequences.empty(); - } - ).when(dataServerClient).run(any(), any(), any()); + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); + return Sequences.empty(); + }).when(dataServerClient).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); Assert.assertThrows(IOE.class, () -> diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 0ac555efe9dc..a72a62f884db 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; @@ -73,7 +74,7 @@ public DataServerClient( this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); } - public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType) + public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType, Closer closer) { final String basePath = "/druid/v2/"; final String cancelPath = basePath + query.getId(); @@ -92,6 +93,7 @@ public Sequence run(Query query, ResponseContext responseContext, Java new DataServerResponseHandler(query, responseContext, objectMapper) ); + closer.register(() -> resultStreamFuture.cancel(true)); Futures.addCallback( resultStreamFuture, new FutureCallback() diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index b7569982056b..57e622ca5eb3 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; @@ -107,7 +108,8 @@ public void testFetchSegmentFromDataServer() throws JsonProcessingException Sequence result = target.run( query, responseContext, - jsonMapper.getTypeFactory().constructType(ScanResultValue.class) + jsonMapper.getTypeFactory().constructType(ScanResultValue.class), + Closer.create() ); Assert.assertEquals(ImmutableList.of(scanResultValue), result.toList()); From fb274799d6edf96b3f4d76fa1bb70b4b7e8da00f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 4 Oct 2023 13:37:25 +0530 Subject: [PATCH 29/37] Resolve test failures --- .../msq/input/table/SegmentWithDescriptor.java | 15 +++++++++------ .../msq/exec/LoadedSegmentDataProviderTest.java | 2 +- .../input/table/SegmentWithDescriptorTest.java | 7 ++++++- 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 191b2d87eed8..857d908f79ce 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; @@ -31,6 +32,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.utils.CollectionUtils; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Objects; import java.util.function.Function; @@ -42,6 +44,7 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; + @Nullable private final LoadedSegmentDataProvider loadedSegmentDataProvider; private final RichSegmentDescriptor descriptor; @@ -56,7 +59,7 @@ public class SegmentWithDescriptor */ public SegmentWithDescriptor( final Supplier> segmentSupplier, - final LoadedSegmentDataProvider loadedSegmentDataProvider, + final @Nullable LoadedSegmentDataProvider loadedSegmentDataProvider, final RichSegmentDescriptor descriptor ) { @@ -86,6 +89,9 @@ public Pair Date: Thu, 5 Oct 2023 23:22:52 +0530 Subject: [PATCH 30/37] Fix bug with certain groupby queries --- .../msq/exec/LoadedSegmentDataProvider.java | 17 +++- .../LoadedSegmentDataProviderFactory.java | 10 ++- .../msq/indexing/IndexerWorkerContext.java | 5 +- .../input/table/SegmentWithDescriptor.java | 3 +- .../GroupByPreShuffleFrameProcessor.java | 13 +-- .../scan/ScanQueryFrameProcessor.java | 1 - .../exec/LoadedSegmentDataProviderTest.java | 18 ++-- .../druid/msq/exec/MSQLoadedSegmentTests.java | 8 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 2 +- .../druid/query/groupby/GroupingEngine.java | 83 ++++++++++--------- 10 files changed, 90 insertions(+), 70 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 760a3f7456fd..963068833c48 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -40,8 +40,11 @@ import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.rpc.FixedSetServiceLocator; @@ -67,13 +70,15 @@ public class LoadedSegmentDataProvider private final ServiceClientFactory serviceClientFactory; private final CoordinatorClient coordinatorClient; private final ObjectMapper objectMapper; + private final QueryToolChestWarehouse warehouse; public LoadedSegmentDataProvider( String dataSource, ChannelCounters channelCounters, ServiceClientFactory serviceClientFactory, CoordinatorClient coordinatorClient, - ObjectMapper objectMapper + ObjectMapper objectMapper, + QueryToolChestWarehouse warehouse ) { this.dataSource = dataSource; @@ -81,6 +86,7 @@ public LoadedSegmentDataProvider( this.serviceClientFactory = serviceClientFactory; this.coordinatorClient = coordinatorClient; this.objectMapper = objectMapper; + this.warehouse = warehouse; } @VisibleForTesting @@ -103,7 +109,6 @@ public Pair> fetchR Query query, RichSegmentDescriptor segmentDescriptor, Function, Sequence> mappingFunction, - Class resultClass, Closer closer ) throws IOException { @@ -114,8 +119,11 @@ public Pair> fetchR Set servers = segmentDescriptor.getServers(); DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); + QueryToolChest> toolChest = warehouse.getToolChest(query); + Function preComputeManipulatorFn = + toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); - final JavaType queryResultType = objectMapper.getTypeFactory().constructType(resultClass); + final JavaType queryResultType = toolChest.getBaseResultType(); final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); @@ -125,7 +133,8 @@ public Pair> fetchR try { statusSequencePair = RetryUtils.retry( () -> { - Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer); + Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) + .map(preComputeManipulatorFn); final List missingSegments = getMissingSegments(responseContext); // Only one segment is fetched, so this should be empty if it was fetched if (missingSegments.isEmpty()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 63e5f8bd0ac0..3985e26062df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; public class LoadedSegmentDataProviderFactory @@ -29,17 +30,19 @@ public class LoadedSegmentDataProviderFactory private final CoordinatorClient coordinatorClient; private final ServiceClientFactory serviceClientFactory; private final ObjectMapper objectMapper; + private final QueryToolChestWarehouse warehouse; public LoadedSegmentDataProviderFactory( CoordinatorClient coordinatorClient, ServiceClientFactory serviceClientFactory, - ObjectMapper objectMapper + ObjectMapper objectMapper, + QueryToolChestWarehouse warehouse ) { - this.coordinatorClient = coordinatorClient; this.serviceClientFactory = serviceClientFactory; this.objectMapper = objectMapper; + this.warehouse = warehouse; } public LoadedSegmentDataProvider createLoadedSegmentDataProvider( @@ -52,7 +55,8 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( channelCounters, serviceClientFactory, coordinatorClient, - objectMapper + objectMapper, + warehouse ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index b237babf6df6..45066baaf25b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -45,6 +45,7 @@ import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocations; import org.apache.druid.rpc.ServiceLocator; @@ -105,6 +106,7 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to final ServiceClientFactory serviceClientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class)); + final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class); return new IndexerWorkerContext( toolbox, @@ -118,7 +120,8 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to new LoadedSegmentDataProviderFactory( toolbox.getCoordinatorClient(), serviceClientFactory, - smileMapper + smileMapper, + warehouse ), serviceClientFactory ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 857d908f79ce..7da381cda7c1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -85,14 +85,13 @@ public ResourceHolder getOrLoad() public Pair> fetchRowsFromDataServer( Query query, Function, Sequence> mappingFunction, - Class queryResultType, Closer closer ) throws IOException { if (loadedSegmentDataProvider == null) { throw new RE("loadedSegmentDataProvider was null. Fetching segments from servers is not supported for segment[%s]", descriptor); } - return loadedSegmentDataProvider.fetchRowsFromDataServer(query, descriptor, mappingFunction, queryResultType, closer); + return loadedSegmentDataProvider.fetchRowsFromDataServer(query, descriptor, mappingFunction, closer); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 3131f5288676..1e9eedc4c436 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -19,7 +19,6 @@ package org.apache.druid.msq.querykit.groupby; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.Frame; @@ -46,7 +45,6 @@ import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; @@ -101,21 +99,12 @@ public GroupByPreShuffleFrameProcessor( ); } - private GroupByQuery prepareGroupByQuery(GroupByQuery query) - { - ImmutableMap overridingContext = ImmutableMap.builder() - .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, query.isApplyLimitPushDown()) - .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) - .build(); - return query.withOverriddenContext(overridingContext); - } - @Override protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { Pair> statusSequencePair = - segment.fetchRowsFromDataServer(prepareGroupByQuery(query), Function.identity(), ResultRow.class, closer); + segment.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), closer); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.", segment.getDescriptor()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 5aff5484c4a5..f03e826696b0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -180,7 +180,6 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s segment.fetchRowsFromDataServer( query, ScanQueryFrameProcessor::mappingFunction, - ScanResultValue.class, closer ); if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java index 4b987fcf7849..c6163aa8663b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java @@ -36,9 +36,14 @@ import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.InputNumberDataSource; import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor; +import org.apache.druid.query.MapQueryToolChestWarehouse; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanQueryQueryToolChest; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.ServiceClientFactory; @@ -105,15 +110,21 @@ public void setUp() .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003/2004")))) .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)) + .context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 1)) .build(); + QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse( + ImmutableMap., QueryToolChest>builder() + .put(ScanQuery.class, new ScanQueryQueryToolChest(null, null)) + .build() + ); target = spy( new LoadedSegmentDataProvider( DATASOURCE1, new ChannelCounters(), mock(ServiceClientFactory.class), coordinatorClient, - DruidServiceTestUtils.newJsonMapper() + DruidServiceTestUtils.newJsonMapper(), + queryToolChestWarehouse ) ); doReturn(dataServerClient).when(target).makeDataServerClient(any()); @@ -128,7 +139,6 @@ public void testFetchRowsFromServer() throws IOException query, SEGMENT_1, ScanQueryFrameProcessor::mappingFunction, - ScanResultValue.class, Closer.create() ); @@ -157,7 +167,6 @@ public void testHandoff() throws IOException query, SEGMENT_1, ScanQueryFrameProcessor::mappingFunction, - ScanResultValue.class, Closer.create() ); @@ -181,7 +190,6 @@ public void testQueryFail() query, SEGMENT_1, ScanQueryFrameProcessor::mappingFunction, - ScanResultValue.class, Closer.create() ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index cdc8fbdf714f..b2c07e267e4c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -114,7 +114,7 @@ public void testSelectWithLoadedSegmentsOnFoo() throws IOException ) ) .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + .fetchRowsFromDataServer(any(), any(), any(), any()); testSelectQuery() .setSql("select cnt, dim1 from foo") @@ -168,7 +168,7 @@ public void testGroupByWithLoadedSegmentsOnFoo() throws IOException ) ) .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + .fetchRowsFromDataServer(any(), any(), any(), any()); testSelectQuery() .setSql("select cnt,count(*) as cnt1 from foo group by cnt") @@ -220,7 +220,7 @@ public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException ImmutableList.of( ResultRow.of(1L, 2L))))) ).when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + .fetchRowsFromDataServer(any(), any(), any(), any()); testSelectQuery() .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") @@ -269,7 +269,7 @@ public void testDataServerQueryFailedShouldFail() throws IOException new ISE("Segment could not be found on data server, but segment was not handed off.") ) .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any(), any()); + .fetchRowsFromDataServer(any(), any(), any(), any()); testSelectQuery() .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 13f7d1e8862a..61ad1f5222bb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -196,7 +196,7 @@ private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProvider LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); try { doThrow(new AssertionError("Test does not support loaded segment query")) - .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any(), any()); + .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any()); doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); } catch (IOException e) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index b79c4358a3de..b242ff98555a 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -177,40 +177,8 @@ public BinaryOperator createMergeFn(Query queryParam) return new GroupByBinaryFnV2((GroupByQuery) queryParam); } - /** - * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are - * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent - * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge - * the rows, apply PostAggregators, and return the resulting {@link Sequence}. - * - * The query will be modified before passing it down to the base runner. For example, "having" clauses will be - * removed and various context parameters will be adjusted. - * - * Despite the similar name, this method is much reduced in scope compared to - * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points, - * but has a truckload of other responsibility, including computing outer query results (if there are subqueries), - * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec. - * - * @param baseRunner base query runner - * @param query the groupBy query to run inside the base query runner - * @param responseContext the response context to pass to the base query runner - * - * @return merged result sequence - */ - public Sequence mergeResults( - final QueryRunner baseRunner, - final GroupByQuery query, - final ResponseContext responseContext - ) + public GroupByQuery prepareGroupByQuery(GroupByQuery query) { - // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may - // involve materialization) - final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( - baseRunner, - this::createResultComparator, - this::createMergeFn - ); - // Set up downstream context. final ImmutableMap.Builder context = ImmutableMap.builder(); context.put(QueryContexts.FINALIZE_KEY, false); @@ -224,7 +192,6 @@ public Sequence mergeResults( final boolean hasTimestampResultField = (timestampResultField != null && !timestampResultField.isEmpty()) && queryContext.getBoolean(CTX_KEY_OUTERMOST, true) && !query.isApplyLimitPushDown(); - int timestampResultFieldIndex = 0; if (hasTimestampResultField) { // sql like "group by city_id,time_floor(__time to day)", // the original translated query is granularity=all and dimensions:[d0, d1] @@ -257,7 +224,7 @@ public Sequence mergeResults( granularity = timestampResultFieldGranularity; // when timestampResultField is the last dimension, should set sortByDimsFirst=true, // otherwise the downstream is sorted by row's timestamp first which makes the final ordering not as expected - timestampResultFieldIndex = queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX); + int timestampResultFieldIndex = queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX, 0); if (!query.getContextSortByDimsFirst() && timestampResultFieldIndex == query.getDimensions().size() - 1) { context.put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true); } @@ -269,7 +236,6 @@ public Sequence mergeResults( // when hasTimestampResultField=true and timestampResultField is neither first nor last dimension, // the DefaultLimitSpec will always do the reordering } - final int timestampResultFieldIndexInOriginalDimensions = timestampResultFieldIndex; if (query.getUniversalTimestamp() != null && !hasTimestampResultField) { // universalTimestamp works only when granularity is all // hasTimestampResultField works only when granularity is all @@ -283,7 +249,7 @@ public Sequence mergeResults( // Always request array result rows when passing the query downstream. context.put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true); - final GroupByQuery newQuery = new GroupByQuery( + return new GroupByQuery( query.getDataSource(), query.getQuerySegmentSpec(), query.getVirtualColumns(), @@ -305,6 +271,49 @@ public Sequence mergeResults( ).withOverriddenContext( context.build() ); + } + + /** + * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are + * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent + * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge + * the rows, apply PostAggregators, and return the resulting {@link Sequence}. + * + * The query will be modified using {@link #prepareGroupByQuery(GroupByQuery)} before passing it down to the base + * runner. For example, "having" clauses will be removed and various context parameters will be adjusted. + * + * Despite the similar name, this method is much reduced in scope compared to + * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points, + * but has a truckload of other responsibility, including computing outer query results (if there are subqueries), + * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec. + * + * @param baseRunner base query runner + * @param query the groupBy query to run inside the base query runner + * @param responseContext the response context to pass to the base query runner + * + * @return merged result sequence + */ + public Sequence mergeResults( + final QueryRunner baseRunner, + final GroupByQuery query, + final ResponseContext responseContext + ) + { + // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may + // involve materialization) + final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( + baseRunner, + this::createResultComparator, + this::createMergeFn + ); + + final QueryContext queryContext = query.context(); + final String timestampResultField = queryContext.getString(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD); + final boolean hasTimestampResultField = (timestampResultField != null && !timestampResultField.isEmpty()) + && queryContext.getBoolean(CTX_KEY_OUTERMOST, true) + && !query.isApplyLimitPushDown(); + final int timestampResultFieldIndexInOriginalDimensions = hasTimestampResultField ? queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX) : 0; + final GroupByQuery newQuery = prepareGroupByQuery(query); final Sequence mergedResults = mergingQueryRunner.run(QueryPlus.wrap(newQuery), responseContext); From feec36f52af67df00826d7371bdb5cef7814e804 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 5 Oct 2023 23:52:44 +0530 Subject: [PATCH 31/37] Resolve unclean merge --- .../druid/msq/querykit/scan/ScanQueryFrameProcessor.java | 6 +++--- .../org/apache/druid/query/IterableRowsCursorHelper.java | 4 +++- .../apache/druid/query/IterableRowsCursorHelperTest.java | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index f03e826696b0..87d68c66c1f1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -65,7 +65,6 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.RowBasedCursor; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; @@ -79,6 +78,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -189,12 +189,12 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s } RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper); - RowBasedCursor cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( + Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( statusSequencePair.rhs, rowSignature ); - final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable))); + final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable.lhs))); if (cursorYielder.isDone()) { // No cursors! diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java index 42d74bcd2189..4bf1cb92a610 100644 --- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java +++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowBasedCursor; @@ -32,6 +33,7 @@ import org.apache.druid.segment.column.RowSignature; import java.io.Closeable; +import java.util.Iterator; /** * Helper methods to create cursor from iterable of rows @@ -83,7 +85,7 @@ public static Pair getCursorFromSequence(Sequence r return Pair.of(baseCursor, rowWalker); } - public static RowBasedCursor getCursorFromYielder(Yielder yielderParam, RowSignature rowSignature) + public static Pair getCursorFromYielder(Yielder yielderParam, RowSignature rowSignature) { return getCursorFromIterable( new Iterable() diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java index 45ba6b8614b7..7628c3289dd1 100644 --- a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java +++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java @@ -64,7 +64,7 @@ public void getCursorFromSequence() @Test public void getCursorFromYielder() { - Cursor cursor = IterableRowsCursorHelper.getCursorFromYielder(Yielders.each(Sequences.simple(rows)), rowSignature); + Cursor cursor = IterableRowsCursorHelper.getCursorFromYielder(Yielders.each(Sequences.simple(rows)), rowSignature).lhs; testCursorMatchesRowSequence(cursor, rowSignature, rows); } From d295b36a4ec5ff9589f13b8a2b8e1c8dd68376df Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 7 Oct 2023 15:19:20 +0530 Subject: [PATCH 32/37] Address review comments --- docs/multi-stage-query/known-issues.md | 2 - docs/multi-stage-query/reference.md | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 16 ++++-- .../msq/exec/LoadedSegmentDataProvider.java | 56 +++++++++++++------ .../apache/druid/msq/exec/SegmentSource.java | 11 +++- .../input/table/DataSegmentWithLocation.java | 2 +- .../input/table/RichSegmentDescriptor.java | 16 +++++- .../input/table/SegmentWithDescriptor.java | 14 ++--- .../input/table/SegmentsInputSliceReader.java | 2 +- .../msq/querykit/BaseLeafFrameProcessor.java | 2 +- .../druid/discovery/DataServerClient.java | 10 ++-- .../discovery/DataServerResponseHandler.java | 4 ++ .../org/apache/druid/rpc/ServiceLocation.java | 16 +++--- .../coordination/DruidServerMetadata.java | 1 + 14 files changed, 101 insertions(+), 53 deletions(-) diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md index bccb9779a835..62a31ecf41af 100644 --- a/docs/multi-stage-query/known-issues.md +++ b/docs/multi-stage-query/known-issues.md @@ -39,8 +39,6 @@ an [UnknownError](./reference.md#error_UnknownError) with a message including "N ## `SELECT` Statement -- `SELECT` from a Druid datasource does not include unpublished real-time data. - - `GROUPING SETS` and `UNION ALL` are not implemented. Queries using these features return a [QueryNotSupported](reference.md#error_QueryNotSupported) error. diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 41467e4defc2..9448abaaa4d4 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -247,7 +247,7 @@ The following table lists the context parameters for the MSQ task engine: | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | | `selectDestination` | SELECT

Controls where the final result of the select query is written.
Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results
Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` | | `waitTillSegmentsLoad` | INSERT, REPLACE

If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` | -| `includeSegmentSource` | SELECT, INSERT, REPLACE

Controls the sources, which will be queried for results, besides the segments in deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only published used segments will be considered. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` | +| `includeSegmentSource` | SELECT, INSERT, REPLACE

Controls the sources, which will be queried for results, besides the segments in deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only non-realtime (published and used) segments will be considered. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` | ## Joins diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 556045b78841..58768644bf69 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1178,23 +1178,27 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() return (dataSource, intervals) -> { final Iterable realtimeAndHistoricalSegments; - // Fetch the realtime segments first, so that we don't miss any segment if they get handed off between the two - // calls. Segments loaded on historicals are also returned here, we deduplicate it below. + // Fetch the realtime segments and segments loaded on the historical. Do this first so that we don't miss any + // segment if they get handed off between the two calls. Segments loaded on historicals are deduplicated below, + // since we are only interested in realtime segments for now. if (includeRealtime) { realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); } else { realtimeAndHistoricalSegments = ImmutableList.of(); } - // Fetch all published used segments from the metadata store. + // Fetch all published, used segments (all non-realtime segments) from the metadata store. final Collection publishedUsedSegments = FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true); int realtimeCount = 0; - // Deduplicate segments, giving preference to metadata store segments. We do this so that if any segments have been - // handed off in between the two metadata calls above, we directly fetch it from deep storage. + // Deduplicate segments, giving preference to published used segments. + // We do this so that if any segments have been handed off in between the two metadata calls above, + // we directly fetch it from deep storage. Set unifiedSegmentView = new HashSet<>(publishedUsedSegments); + + // Iterate over the realtime segments and segments loaded on the historical for (ImmutableSegmentLoadInfo segmentLoadInfo : realtimeAndHistoricalSegments) { ImmutableSet servers = segmentLoadInfo.getServers(); // Filter out only realtime servers. We don't want to query historicals for now, but we can in the future. @@ -1212,6 +1216,8 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() realtimeServerMetadata ); unifiedSegmentView.add(dataSegmentWithLocation); + } else { + // We don't have any segments of the required segment source, ignore the segment } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 963068833c48..fdca3dc3e47a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -27,6 +27,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; @@ -44,6 +45,7 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; @@ -55,6 +57,7 @@ import java.io.IOException; import java.util.List; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.function.Function; /** @@ -96,11 +99,20 @@ DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) } /** - * Queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver indicates - * that the segment was not found, checks with the coordinator to see if the segment was handed off. + * Performs some necessary transforms to the query, so that the dataserver is able to understand it first. + * - Changing the datasource to a {@link TableDataSource} + * - Limiting the query to a single required segment with {@link Queries#withSpecificSegments(Query, List)} + *
+ * Then queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver + * indicates that the segment was not found, checks with the coordinator to see if the segment was handed off. * - If the segment was handed off, returns with a {@link DataServerQueryStatus#HANDOFF} status. * - If the segment was not handed off, retries with the known list of servers and throws an exception if the retry * count is exceeded. + * - If the servers could not be found, checks if the segment was handed-off. If it was, returns with a + * {@link DataServerQueryStatus#HANDOFF} status. Otherwise, throws an exception. + *
+ * Also applies {@link QueryToolChest#makePreComputeManipulatorFn(Query, MetricManipulationFn)} and reports channel + * metrics on the returned results. * * @param result return type for the query from the data server * @param type of the result rows after parsing from QueryType object @@ -117,10 +129,10 @@ public Pair> fetchR ImmutableList.of(segmentDescriptor) ); - Set servers = segmentDescriptor.getServers(); - DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); - QueryToolChest> toolChest = warehouse.getToolChest(query); - Function preComputeManipulatorFn = + final Set servers = segmentDescriptor.getServers(); + final DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); + final QueryToolChest> toolChest = warehouse.getToolChest(query); + final Function preComputeManipulatorFn = toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); final JavaType queryResultType = toolChest.getBaseResultType(); @@ -131,6 +143,8 @@ public Pair> fetchR Pair> statusSequencePair; try { + // We need to check for handoff to decide if we need to retry. Therefore, we handle it here instead of inside + // the client. statusSequencePair = RetryUtils.retry( () -> { Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) @@ -171,21 +185,29 @@ public Pair> fetchR catch (QueryInterruptedException e) { if (e.getCause() instanceof RpcException) { // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), - // we would also be unable to fetch the segment. Check if the segment was handed off, just in case. + // we would also be unable to fetch the segment. Check if the segment was handed off, just in case, instead of + // failing the query. boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); if (wasHandedOff) { log.debug("Segment[%s] was handed off.", segmentDescriptor); return Pair.of(DataServerQueryStatus.HANDOFF, null); } } - throw new IOE(e, "Exception while fetching rows for query from dataservers[%s]", servers); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception while fetching rows for query from dataservers[%s]", servers); } catch (Exception e) { Throwables.propagateIfPossible(e, IOE.class); - throw new IOE(e, "Exception while fetching rows for query from dataservers[%s]", servers); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception while fetching rows for query from dataservers[%s]", servers); } } + /** + * Retreives the list of missing segments from the response context. + */ private static List getMissingSegments(final ResponseContext responseContext) { List missingSegments = responseContext.getMissingSegments(); @@ -195,6 +217,11 @@ private static List getMissingSegments(final ResponseContext return missingSegments; } + /** + * Queries the coordinator to check if a segment has been handed off. + *
+ * See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)} + */ private static boolean checkSegmentHandoff( CoordinatorClient coordinatorClient, String dataSource, @@ -203,16 +230,11 @@ private static boolean checkSegmentHandoff( { Boolean wasHandedOff; try { - wasHandedOff = RetryUtils.retry( - () -> FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true), - input -> true, - DEFAULT_NUM_TRIES - ); + wasHandedOff = FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true); } - catch (Exception e) { - throw new IOE(e, "Could not contact coordinator"); + catch (InterruptedException | ExecutionException e) { + throw new IOE(e, "Could not contact coordinator for segment[%s]", segmentDescriptor); } - return Boolean.TRUE.equals(wasHandedOff); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java index 5e61ce9c6ac7..07b56f5f8549 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java @@ -24,12 +24,18 @@ import java.util.Set; +/** + * Decides the types of data servers contacted by MSQ tasks to fetch results. + */ public enum SegmentSource { - NONE(ImmutableSet.of()), REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR)); + /** + * The type of dataservers (if any) to include. This does not include segments queried from deep storage, which are + * always included in queries. + */ private final Set usedServerTypes; SegmentSource(Set usedServerTypes) @@ -42,6 +48,9 @@ public Set getUsedServerTypes() return usedServerTypes; } + /** + * Whether realtime servers should be included for the segmentSource. + */ public static boolean shouldQueryRealtimeServers(SegmentSource segmentSource) { return REALTIME.equals(segmentSource); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java index afc006902e89..0e83e9c3edee 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -90,7 +90,7 @@ public DataSegmentWithLocation( this.servers = servers; } - @JsonProperty + @JsonProperty("servers") public Set getServers() { return servers; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index ed8c8f7ff826..04e4e601b073 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -90,6 +91,14 @@ static RichSegmentDescriptor fromJson( ); } + /** + * Returns true if the location the segment is loaded is available, and false if it is not. + */ + public boolean isLoadedOnServer() + { + return !CollectionUtils.isNullOrEmpty(getServers()); + } + @JsonProperty("servers") public Set getServers() { @@ -135,8 +144,11 @@ public int hashCode() public String toString() { return "RichSegmentDescriptor{" + - "fullInterval=" + fullInterval + - ", servers=" + servers + + "fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) + + ", servers=" + getServers() + + ", interval=" + getInterval() + + ", version='" + getVersion() + '\'' + + ", partitionNumber=" + getPartitionNumber() + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 7da381cda7c1..137129ed338b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -21,16 +21,14 @@ import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.LoadedSegmentDataProvider; import org.apache.druid.query.Query; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.Segment; -import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -89,7 +87,8 @@ public Pair Pair dataSegmentIterator( return new SegmentWithDescriptor( dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), - loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, channelCounters), + descriptor.isLoadedOnServer() ? loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, channelCounters) : null, descriptor ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 7361e1ba801a..f67f30d0c5c6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -83,7 +83,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throw if (baseInput.hasSegment()) { SegmentWithDescriptor segment = baseInput.getSegment(); - if (segment.isLoadedOnServer()) { + if (segment.getDescriptor().isLoadedOnServer()) { retVal = runWithLoadedSegment(baseInput.getSegment()); } else { retVal = runWithSegment(baseInput.getSegment()); diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index a72a62f884db..80eff3a947f4 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -54,6 +54,7 @@ */ public class DataServerClient { + private static final String BASE_PATH = "/druid/v2/"; private static final Logger log = new Logger(DataServerClient.class); private final ServiceClient serviceClient; private final ObjectMapper objectMapper; @@ -76,10 +77,9 @@ public DataServerClient( public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType, Closer closer) { - final String basePath = "/druid/v2/"; - final String cancelPath = basePath + query.getId(); + final String cancelPath = BASE_PATH + query.getId(); - RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, basePath); + RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, BASE_PATH); final boolean isSmile = objectMapper.getFactory() instanceof SmileFactory; if (isSmile) { requestBuilder = requestBuilder.smileContent(objectMapper, query); @@ -124,9 +124,9 @@ public JsonParserIterator make() return new JsonParserIterator<>( queryResultType, resultStreamFuture, - basePath, + BASE_PATH, query, - "", + "", // TODO: this objectMapper ); } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index 6c8968cd87e9..7715000f8209 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -34,6 +34,10 @@ import java.io.IOException; import java.io.InputStream; +/** + * Response handler for the {@link DataServerClient}. Handles the input stream from the data server and handles updating + * the {@link ResponseContext} from the header. Does not apply backpressure or query timeout. + */ public class DataServerResponseHandler implements HttpResponseHandler { private static final Logger log = new Logger(DataServerResponseHandler.class); diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java index 6e20cdca6076..3a092d7cb8dd 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -26,7 +26,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; -import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.util.Iterator; import java.util.Objects; @@ -57,18 +57,20 @@ public static ServiceLocation fromDruidNode(final DruidNode druidNode) public static ServiceLocation fromDruidServerMetadata(final DruidServerMetadata druidServerMetadata) { - final String host = getHostFromString(druidServerMetadata.getHost()); + final String host = getHostFromString( + Preconditions.checkNotNull( + druidServerMetadata.getHost(), + "Host was null for druid server metadata[%s]", + druidServerMetadata + ) + ); int plaintextPort = getPortFromString(druidServerMetadata.getHostAndPort()); int tlsPort = getPortFromString(druidServerMetadata.getHostAndTlsPort()); return new ServiceLocation(host, plaintextPort, tlsPort, ""); } - @Nullable - private static String getHostFromString(String s) + private static String getHostFromString(@NotNull String s) { - if (s == null) { - return null; - } Iterator iterator = SPLITTER.split(s).iterator(); ImmutableList strings = ImmutableList.copyOf(iterator); return strings.get(0); diff --git a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java index 1a8a4388f8d8..fcb08d26a2bf 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java @@ -40,6 +40,7 @@ public class DruidServerMetadata private final ServerType type; private final int priority; + // Either hostAndPort or hostAndTlsPort would be null depending on the type of connection. @JsonCreator public DruidServerMetadata( @JsonProperty("name") String name, From 69eb9c8d41f1413e0eddd1cdf0eb98e0b9663fbf Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 7 Oct 2023 22:15:20 +0530 Subject: [PATCH 33/37] Address review comments --- .../apache/druid/msq/exec/LoadedSegmentDataProvider.java | 8 ++++++-- .../druid/msq/exec/LoadedSegmentDataProviderFactory.java | 8 +++++++- .../druid/msq/exec/LoadedSegmentDataProviderTest.java | 4 +++- .../org/apache/druid/discovery/DataServerClient.java | 9 ++++++--- .../org/apache/druid/rpc/FixedSetServiceLocator.java | 5 +++++ .../org/apache/druid/discovery/DataServerClientTest.java | 4 +++- 6 files changed, 30 insertions(+), 8 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index fdca3dc3e47a..2139c1329c1d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -58,6 +58,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; /** @@ -74,6 +75,7 @@ public class LoadedSegmentDataProvider private final CoordinatorClient coordinatorClient; private final ObjectMapper objectMapper; private final QueryToolChestWarehouse warehouse; + private final ScheduledExecutorService queryCancellationExecutor; public LoadedSegmentDataProvider( String dataSource, @@ -81,7 +83,8 @@ public LoadedSegmentDataProvider( ServiceClientFactory serviceClientFactory, CoordinatorClient coordinatorClient, ObjectMapper objectMapper, - QueryToolChestWarehouse warehouse + QueryToolChestWarehouse warehouse, + ScheduledExecutorService queryCancellationExecutor ) { this.dataSource = dataSource; @@ -90,12 +93,13 @@ public LoadedSegmentDataProvider( this.coordinatorClient = coordinatorClient; this.objectMapper = objectMapper; this.warehouse = warehouse; + this.queryCancellationExecutor = queryCancellationExecutor; } @VisibleForTesting DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) { - return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper); + return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper, queryCancellationExecutor); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 3985e26062df..2db753cf5a11 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -21,16 +21,20 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; +import java.util.concurrent.ScheduledExecutorService; + public class LoadedSegmentDataProviderFactory { private final CoordinatorClient coordinatorClient; private final ServiceClientFactory serviceClientFactory; private final ObjectMapper objectMapper; private final QueryToolChestWarehouse warehouse; + private final ScheduledExecutorService queryCancellationExecutor; public LoadedSegmentDataProviderFactory( CoordinatorClient coordinatorClient, @@ -43,6 +47,7 @@ public LoadedSegmentDataProviderFactory( this.serviceClientFactory = serviceClientFactory; this.objectMapper = objectMapper; this.warehouse = warehouse; + this.queryCancellationExecutor = ScheduledExecutors.fixed(10, "query-cancellation-executor"); } public LoadedSegmentDataProvider createLoadedSegmentDataProvider( @@ -56,7 +61,8 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( serviceClientFactory, coordinatorClient, objectMapper, - warehouse + warehouse, + queryCancellationExecutor ); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java index c6163aa8663b..bcfd5fb42120 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.io.Closer; @@ -124,7 +125,8 @@ public void setUp() mock(ServiceClientFactory.class), coordinatorClient, DruidServiceTestUtils.newJsonMapper(), - queryToolChestWarehouse + queryToolChestWarehouse, + Execs.scheduledSingleThreaded("query-cancellation-executor") ) ); doReturn(dataServerClient).when(target).makeDataServerClient(any()); diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 80eff3a947f4..6aba509c83b5 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -58,12 +58,14 @@ public class DataServerClient private static final Logger log = new Logger(DataServerClient.class); private final ServiceClient serviceClient; private final ObjectMapper objectMapper; + private final FixedSetServiceLocator fixedSetServiceLocator; private final ScheduledExecutorService queryCancellationExecutor; public DataServerClient( ServiceClientFactory serviceClientFactory, FixedSetServiceLocator fixedSetServiceLocator, - ObjectMapper objectMapper + ObjectMapper objectMapper, + ScheduledExecutorService queryCancellationExecutor ) { this.serviceClient = serviceClientFactory.makeClient( @@ -71,8 +73,9 @@ public DataServerClient( fixedSetServiceLocator, StandardRetryPolicy.noRetries() ); + this.fixedSetServiceLocator = fixedSetServiceLocator; this.objectMapper = objectMapper; - this.queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor"); + this.queryCancellationExecutor = queryCancellationExecutor; } public Sequence run(Query query, ResponseContext responseContext, JavaType queryResultType, Closer closer) @@ -126,7 +129,7 @@ public JsonParserIterator make() resultStreamFuture, BASE_PATH, query, - "", // TODO: this + fixedSetServiceLocator.getServiceLocations().toString(), // For logging objectMapper ); } diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java index a5149ceb7346..de99acb4b7b9 100644 --- a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -53,6 +53,11 @@ public static FixedSetServiceLocator forDruidServerMetadata(Set locate() { diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index 57e622ca5eb3..2c01861fe84b 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.SegmentDescriptor; @@ -80,7 +81,8 @@ public void setUp() target = new DataServerClient( serviceClientFactory, mock(FixedSetServiceLocator.class), - jsonMapper + jsonMapper, + Execs.scheduledSingleThreaded("query-cancellation-executor") ); } From d2356180ebed9be87eeebbbdbeb8e60c63cf929f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 8 Oct 2023 16:01:41 +0530 Subject: [PATCH 34/37] Address review comments --- .../msq/exec/LoadedSegmentDataProvider.java | 10 +++++-- .../LoadedSegmentDataProviderFactory.java | 30 +++++++++++++++++-- .../apache/druid/msq/exec/WorkerContext.java | 1 + .../org/apache/druid/msq/exec/WorkerImpl.java | 1 + .../msq/indexing/IndexerWorkerContext.java | 6 ++++ .../druid/msq/test/MSQTestWorkerContext.java | 6 ++++ .../druid/discovery/DataServerClient.java | 11 +++---- .../druid/rpc/FixedSetServiceLocator.java | 11 +++---- .../druid/discovery/DataServerClientTest.java | 4 +-- 9 files changed, 63 insertions(+), 17 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 2139c1329c1d..4bf6b8bfa059 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; @@ -52,6 +53,7 @@ import org.apache.druid.rpc.FixedSetServiceLocator; import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; import java.io.IOException; @@ -97,9 +99,9 @@ public LoadedSegmentDataProvider( } @VisibleForTesting - DataServerClient makeDataServerClient(FixedSetServiceLocator serviceLocator) + DataServerClient makeDataServerClient(ServiceLocation serviceLocation) { - return new DataServerClient(serviceClientFactory, serviceLocator, objectMapper, queryCancellationExecutor); + return new DataServerClient(serviceClientFactory, serviceLocation, objectMapper, queryCancellationExecutor); } /** @@ -134,7 +136,7 @@ public Pair> fetchR ); final Set servers = segmentDescriptor.getServers(); - final DataServerClient dataServerClient = makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers)); + final FixedSetServiceLocator fixedSetServiceLocator = FixedSetServiceLocator.forDruidServerMetadata(servers); final QueryToolChest> toolChest = warehouse.getToolChest(query); final Function preComputeManipulatorFn = toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); @@ -151,6 +153,8 @@ public Pair> fetchR // the client. statusSequencePair = RetryUtils.retry( () -> { + ServiceLocation serviceLocation = Iterables.getOnlyElement(fixedSetServiceLocator.locate().get().getLocations()); + DataServerClient dataServerClient = makeDataServerClient(serviceLocation); Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) .map(preComputeManipulatorFn); final List missingSegments = getMissingSegments(responseContext); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 2db753cf5a11..99c59a67da8e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -21,15 +21,25 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; +import java.io.Closeable; +import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; -public class LoadedSegmentDataProviderFactory +/** + * Creates new instances of {@link LoadedSegmentDataProvider} and manages the cancellation threadpool. + */ +public class LoadedSegmentDataProviderFactory implements Closeable { + private static final Logger log = new Logger(LoadedSegmentDataProviderFactory.class); + private static final int DEFAULT_THREAD_COUNT = 4; private final CoordinatorClient coordinatorClient; private final ServiceClientFactory serviceClientFactory; private final ObjectMapper objectMapper; @@ -47,7 +57,7 @@ public LoadedSegmentDataProviderFactory( this.serviceClientFactory = serviceClientFactory; this.objectMapper = objectMapper; this.warehouse = warehouse; - this.queryCancellationExecutor = ScheduledExecutors.fixed(10, "query-cancellation-executor"); + this.queryCancellationExecutor = ScheduledExecutors.fixed(DEFAULT_THREAD_COUNT, "query-cancellation-executor"); } public LoadedSegmentDataProvider createLoadedSegmentDataProvider( @@ -65,4 +75,20 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( queryCancellationExecutor ); } + + @Override + public void close() throws IOException + { + // Wait for all query cancellations to be complete. + queryCancellationExecutor.shutdown(); + try { + if (!queryCancellationExecutor.awaitTermination(1, TimeUnit.MINUTES)) { + log.error("Unable to cancel all ongoing queries."); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RE(e); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java index d017feb099fb..a3d4fde6c1a5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -74,6 +74,7 @@ public interface WorkerContext DruidNode selfNode(); Bouncer processorBouncer(); + LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory(); default File tempDir(int stageNumber, String id) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 83808f527c99..6ee45bc158e8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -295,6 +295,7 @@ public Optional runTask(final Closer closer) throws Exception { this.controllerClient = context.makeControllerClient(task.getControllerTaskId()); closer.register(controllerClient::close); + closer.register(context.loadedSegmentDataProviderFactory()); context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 45066baaf25b..709b019891f0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -268,6 +268,12 @@ public Bouncer processorBouncer() return injector.getInstance(Bouncer.class); } + @Override + public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + { + return loadedSegmentDataProviderFactory; + } + private synchronized OverlordClient makeOverlordClient() { if (overlordClient == null) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 88cdf36ffa66..51d83397ccae 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -182,4 +182,10 @@ public Bouncer processorBouncer() { return injector.getInstance(Bouncer.class); } + + @Override + public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + { + return injector.getInstance(LoadedSegmentDataProviderFactory.class); + } } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 6aba509c83b5..0eb5833da479 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -39,6 +39,7 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.utils.CloseableUtils; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -58,22 +59,22 @@ public class DataServerClient private static final Logger log = new Logger(DataServerClient.class); private final ServiceClient serviceClient; private final ObjectMapper objectMapper; - private final FixedSetServiceLocator fixedSetServiceLocator; + private final ServiceLocation serviceLocation; private final ScheduledExecutorService queryCancellationExecutor; public DataServerClient( ServiceClientFactory serviceClientFactory, - FixedSetServiceLocator fixedSetServiceLocator, + ServiceLocation serviceLocation, ObjectMapper objectMapper, ScheduledExecutorService queryCancellationExecutor ) { this.serviceClient = serviceClientFactory.makeClient( NodeRole.INDEXER.getJsonName(), - fixedSetServiceLocator, + FixedSetServiceLocator.forServiceLocation(serviceLocation), StandardRetryPolicy.noRetries() ); - this.fixedSetServiceLocator = fixedSetServiceLocator; + this.serviceLocation = serviceLocation; this.objectMapper = objectMapper; this.queryCancellationExecutor = queryCancellationExecutor; } @@ -129,7 +130,7 @@ public JsonParserIterator make() resultStreamFuture, BASE_PATH, query, - fixedSetServiceLocator.getServiceLocations().toString(), // For logging + serviceLocation.getHost(), objectMapper ); } diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java index de99acb4b7b9..f4bfa18470d2 100644 --- a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -24,6 +24,7 @@ import org.apache.druid.server.coordination.DruidServerMetadata; import org.jboss.netty.util.internal.ThreadLocalRandom; +import javax.validation.constraints.NotNull; import java.util.Set; import java.util.stream.Collectors; @@ -40,6 +41,11 @@ private FixedSetServiceLocator(ServiceLocations serviceLocations) this.serviceLocations = serviceLocations; } + public static FixedSetServiceLocator forServiceLocation(@NotNull ServiceLocation serviceLocation) + { + return new FixedSetServiceLocator(ServiceLocations.forLocation(serviceLocation)); + } + public static FixedSetServiceLocator forDruidServerMetadata(Set serverMetadataSet) { if (serverMetadataSet == null || serverMetadataSet.isEmpty()) { @@ -53,11 +59,6 @@ public static FixedSetServiceLocator forDruidServerMetadata(Set locate() { diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index 2c01861fe84b..7be5a13474d5 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -33,10 +33,10 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; -import org.apache.druid.rpc.FixedSetServiceLocator; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.junit.Assert; @@ -80,7 +80,7 @@ public void setUp() target = new DataServerClient( serviceClientFactory, - mock(FixedSetServiceLocator.class), + mock(ServiceLocation.class), jsonMapper, Execs.scheduledSingleThreaded("query-cancellation-executor") ); From e573e31904588b53f8dce1230af1f9e07ed91066 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 8 Oct 2023 16:41:04 +0530 Subject: [PATCH 35/37] Resolve checkstyle failures --- .../druid/msq/exec/LoadedSegmentDataProviderFactory.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 99c59a67da8e..04c476287722 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -29,7 +29,6 @@ import org.apache.druid.rpc.ServiceClientFactory; import java.io.Closeable; -import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -77,7 +76,7 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( } @Override - public void close() throws IOException + public void close() { // Wait for all query cancellations to be complete. queryCancellationExecutor.shutdown(); From 7ddf0250b16e540c9a56f2d7aa2fe9dcb3dafce8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 9 Oct 2023 11:43:40 +0530 Subject: [PATCH 36/37] Address review comments --- docs/multi-stage-query/reference.md | 2 +- .../msq/exec/LoadedSegmentDataProvider.java | 10 ++-- .../apache/druid/msq/exec/SegmentSource.java | 6 +++ .../scan/ScanQueryFrameProcessor.java | 1 + .../exec/LoadedSegmentDataProviderTest.java | 48 +++++++++++++++++++ .../druid/msq/test/CalciteMSQTestsHelper.java | 3 ++ .../druid/discovery/DataServerClient.java | 2 +- 7 files changed, 66 insertions(+), 6 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 9448abaaa4d4..5e80e318b8c8 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -247,7 +247,7 @@ The following table lists the context parameters for the MSQ task engine: | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | | `selectDestination` | SELECT

Controls where the final result of the select query is written.
Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results
Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` | | `waitTillSegmentsLoad` | INSERT, REPLACE

If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` | -| `includeSegmentSource` | SELECT, INSERT, REPLACE

Controls the sources, which will be queried for results, besides the segments in deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only non-realtime (published and used) segments will be considered. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` | +| `includeSegmentSource` | SELECT, INSERT, REPLACE

Controls the sources, which will be queried for results in addition to the segments present on deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only non-realtime (published and used) segments will be downloaded from deep storage. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` | ## Joins diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 4bf6b8bfa059..734b3b8387ce 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -24,7 +24,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; @@ -55,11 +54,11 @@ import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.utils.CollectionUtils; import java.io.IOException; import java.util.List; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; @@ -153,7 +152,10 @@ public Pair> fetchR // the client. statusSequencePair = RetryUtils.retry( () -> { - ServiceLocation serviceLocation = Iterables.getOnlyElement(fixedSetServiceLocator.locate().get().getLocations()); + ServiceLocation serviceLocation = CollectionUtils.getOnlyElement( + fixedSetServiceLocator.locate().get().getLocations(), + serviceLocations -> { throw DruidException.defensive("Should only have one location"); } + ); DataServerClient dataServerClient = makeDataServerClient(serviceLocation); Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) .map(preComputeManipulatorFn); @@ -240,7 +242,7 @@ private static boolean checkSegmentHandoff( try { wasHandedOff = FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true); } - catch (InterruptedException | ExecutionException e) { + catch (Exception e) { throw new IOE(e, "Could not contact coordinator for segment[%s]", segmentDescriptor); } return Boolean.TRUE.equals(wasHandedOff); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java index 07b56f5f8549..22f3a5df973c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java @@ -29,7 +29,13 @@ */ public enum SegmentSource { + /** + * Include only segments from deep storage. + */ NONE(ImmutableSet.of()), + /** + * Include segments from realtime tasks as well as segments from deep storage. + */ REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR)); /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 87d68c66c1f1..1541d314f215 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -194,6 +194,7 @@ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor s rowSignature ); + closer.register(cursorFromIterable.rhs); final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable.lhs))); if (cursorYielder.isDone()) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java index bcfd5fb42120..6c6ad1b3fa9a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java @@ -26,6 +26,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.discovery.DataServerClient; import org.apache.druid.discovery.DruidServiceTestUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -40,6 +41,7 @@ import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.context.ResponseContext; @@ -47,6 +49,7 @@ import org.apache.druid.query.scan.ScanQueryQueryToolChest; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -63,8 +66,11 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; @RunWith(MockitoJUnitRunner.class) public class LoadedSegmentDataProviderTest @@ -176,6 +182,48 @@ public void testHandoff() throws IOException Assert.assertNull(dataServerQueryStatusYielderPair.rhs); } + @Test + public void testServerNotFoundWithoutHandoffShouldThrowException() + { + doThrow( + new QueryInterruptedException(new RpcException("Could not connect to server")) + ).when(dataServerClient).run(any(), any(), any(), any()); + + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + + ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); + + Assert.assertThrows(DruidException.class, () -> + target.fetchRowsFromDataServer( + queryWithRetry, + SEGMENT_1, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ) + ); + + verify(dataServerClient, times(3)).run(any(), any(), any(), any()); + } + + @Test + public void testServerNotFoundButHandoffShouldReturnWithStatus() throws IOException + { + doThrow( + new QueryInterruptedException(new RpcException("Could not connect to server")) + ).when(dataServerClient).run(any(), any(), any(), any()); + + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + + Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( + query, + SEGMENT_1, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ); + + Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs); + Assert.assertNull(dataServerQueryStatusYielderPair.rhs); + } @Test public void testQueryFail() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 61ad1f5222bb..c68b2331c7d9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -192,6 +192,9 @@ public String getFormatString() private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() { + // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. + // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like + // org.apache.druid.msq.exec.MSQLoadedSegmentTests. LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); try { diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java index 0eb5833da479..479ba9d4142d 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerClient.java @@ -70,7 +70,7 @@ public DataServerClient( ) { this.serviceClient = serviceClientFactory.makeClient( - NodeRole.INDEXER.getJsonName(), + serviceLocation.getHost(), FixedSetServiceLocator.forServiceLocation(serviceLocation), StandardRetryPolicy.noRetries() ); From 1cfe72a34c2297f8ceed9e6ae88d1114b894f23f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 9 Oct 2023 11:50:39 +0530 Subject: [PATCH 37/37] Add log statement --- .../org/apache/druid/msq/exec/LoadedSegmentDataProvider.java | 4 +++- .../druid/msq/exec/LoadedSegmentDataProviderFactory.java | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java index 734b3b8387ce..d9d789e3d2ba 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java @@ -154,7 +154,9 @@ public Pair> fetchR () -> { ServiceLocation serviceLocation = CollectionUtils.getOnlyElement( fixedSetServiceLocator.locate().get().getLocations(), - serviceLocations -> { throw DruidException.defensive("Should only have one location"); } + serviceLocations -> { + throw DruidException.defensive("Should only have one location"); + } ); DataServerClient dataServerClient = makeDataServerClient(serviceLocation); Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java index 04c476287722..48ed57be8701 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java @@ -79,6 +79,7 @@ public LoadedSegmentDataProvider createLoadedSegmentDataProvider( public void close() { // Wait for all query cancellations to be complete. + log.info("Waiting for any data server queries to be canceled."); queryCancellationExecutor.shutdown(); try { if (!queryCancellationExecutor.awaitTermination(1, TimeUnit.MINUTES)) {