cryptoe commented on code in PR #15024:
URL: https://github.com/apache/druid/pull/15024#discussion_r1348254321


##########
docs/multi-stage-query/reference.md:
##########
@@ -247,6 +247,7 @@ The following table lists the context parameters for the 
MSQ task engine:
 | `faultTolerance` | SELECT, INSERT, REPLACE<br /><br /> 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<br /><br /> Controls where the final result of 
the select query is written. <br />Use `taskReport`(the default) to write 
select results to the task report. <b> This is not scalable since task reports 
size explodes for large results </b> <br/>Use `durableStorage` to write results 
to durable storage location. <b>For large results sets, its recommended to use 
`durableStorage` </b>. To configure durable storage see 
[`this`](#durable-storage) section.                                             
                                                                                
                                                                                
                                                                                
                                                                                
                                                                                
                                                             | `taskRep
 ort` |
 | `waitTillSegmentsLoad` | INSERT, REPLACE<br /><br /> 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<br /><br /> 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` |
 

Review Comment:
   Lets update the known issues in MSQ as well in this patch. 



##########
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()),

Review Comment:
   Please java doc these methods. Since null queries deep storage we should 
mention that here. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =
           
FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource,
 intervals), true);
 
-      if (dataSegments.isEmpty()) {
+      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<DataSegment> unifiedSegmentView = new 
HashSet<>(publishedUsedSegments);
+      for (ImmutableSegmentLoadInfo segmentLoadInfo : 
realtimeAndHistoricalSegments) {
+        ImmutableSet<DruidServerMetadata> 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<DruidServerMetadata> realtimeServerMetadata
+            = servers.stream()

Review Comment:
   Lets filter on unifiedSegmentView as well. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =
           
FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource,
 intervals), true);
 
-      if (dataSegments.isEmpty()) {
+      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<DataSegment> unifiedSegmentView = new 
HashSet<>(publishedUsedSegments);
+      for (ImmutableSegmentLoadInfo segmentLoadInfo : 
realtimeAndHistoricalSegments) {

Review Comment:
   Nit: we can add a space here



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java:
##########
@@ -97,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 +

Review Comment:
   We should add 
   ``` "fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) +
              ", interval=" + getInterval() +
              ", version='" + getVersion() + '\'' +
              ", partitionNumber=" + getPartitionNumber() 
              
              ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.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.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.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;
+import org.apache.druid.rpc.RpcException;
+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 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;
+  private final QueryToolChestWarehouse warehouse;
+
+  public LoadedSegmentDataProvider(
+      String dataSource,
+      ChannelCounters channelCounters,
+      ServiceClientFactory serviceClientFactory,
+      CoordinatorClient coordinatorClient,
+      ObjectMapper objectMapper,
+      QueryToolChestWarehouse warehouse
+  )
+  {
+    this.dataSource = dataSource;
+    this.channelCounters = channelCounters;
+    this.serviceClientFactory = serviceClientFactory;
+    this.coordinatorClient = coordinatorClient;
+    this.objectMapper = objectMapper;
+    this.warehouse = warehouse;
+  }
+
+  @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.

Review Comment:
   What happens if the dataserver/indexer is not found. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java:
##########
@@ -45,18 +99,118 @@ public interface LoadedSegmentDataProvider
    * @param <QueryType> result return type for the query from the data server
    * @param <RowType> type of the result rows after parsing from QueryType 
object
    */
-  <RowType, QueryType> Pair<DataServerQueryStatus, Yielder<RowType>> 
fetchRowsFromDataServer(
+  public <RowType, QueryType> Pair<DataServerQueryStatus, Yielder<RowType>> 
fetchRowsFromDataServer(
       Query<QueryType> query,
       RichSegmentDescriptor segmentDescriptor,
       Function<Sequence<QueryType>, Sequence<RowType>> mappingFunction,
-      Class<QueryType> queryResultType,
+      Class<QueryType> resultClass,

Review Comment:
   I guess this might not be needed. 
   



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =

Review Comment:
   Does it make sense to have DataSegmentWithLocation here ?
   Location can be null for segments which are fetched from deep storage 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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;
+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;
+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<DruidServerMetadata> 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<String, Object> loadSpec,
+      @JsonProperty("dimensions")
+      @JsonDeserialize(using = CommaListJoinDeserializer.class)
+      @Nullable
+      List<String> dimensions,
+      @JsonProperty("metrics")
+      @JsonDeserialize(using = CommaListJoinDeserializer.class)
+      @Nullable
+      List<String> metrics,
+      @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec,
+      @JsonProperty("lastCompactionState") @Nullable CompactionState 
lastCompactionState,
+      @JsonProperty("binaryVersion") Integer binaryVersion,
+      @JsonProperty("size") long size,
+      @JsonProperty("servers") Set<DruidServerMetadata> servers,
+      @JacksonInject PruneSpecsHolder pruneSpecsHolder
+  )
+  {
+    super(dataSource, interval, version, loadSpec, dimensions, metrics, 
shardSpec, lastCompactionState, binaryVersion, size, pruneSpecsHolder);
+    this.servers = Preconditions.checkNotNull(servers, "servers");
+  }
+
+  public DataSegmentWithLocation(
+      DataSegment dataSegment,
+      Set<DruidServerMetadata> servers
+  )
+  {
+    super(
+        dataSegment.getDataSource(),
+        dataSegment.getInterval(),
+        dataSegment.getVersion(),
+        dataSegment.getLoadSpec(),
+        dataSegment.getDimensions(),
+        dataSegment.getMetrics(),
+        dataSegment.getShardSpec(),
+        dataSegment.getBinaryVersion(),
+        dataSegment.getSize()
+    );
+    this.servers = servers;
+  }
+
+  @JsonProperty

Review Comment:
   ```suggestion
   @JsonProperty("servers")
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.

Review Comment:
   ```suggestion
         // Fetch all published used segments ie all non realtime segments
   ```



##########
docs/multi-stage-query/reference.md:
##########
@@ -247,6 +247,7 @@ The following table lists the context parameters for the 
MSQ task engine:
 | `faultTolerance` | SELECT, INSERT, REPLACE<br /><br /> 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<br /><br /> Controls where the final result of 
the select query is written. <br />Use `taskReport`(the default) to write 
select results to the task report. <b> This is not scalable since task reports 
size explodes for large results </b> <br/>Use `durableStorage` to write results 
to durable storage location. <b>For large results sets, its recommended to use 
`durableStorage` </b>. To configure durable storage see 
[`this`](#durable-storage) section.                                             
                                                                                
                                                                                
                                                                                
                                                                                
                                                                                
                                                             | `taskRep
 ort` |
 | `waitTillSegmentsLoad` | INSERT, REPLACE<br /><br /> 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<br /><br /> 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` |

Review Comment:
   lets use published used segments or non-realtime segments ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =
           
FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource,
 intervals), true);
 
-      if (dataSegments.isEmpty()) {
+      int realtimeCount = 0;
+
+      // Deduplicate segments, giving preference to metadata store segments. 
We do this so that if any segments have been

Review Comment:
   ```suggestion
         // Deduplicate segments, giving preference to non realtime segments 
since they can be queried from deep storage. We do this so that if any segments 
have been
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java:
##########
@@ -94,6 +98,7 @@ private Iterator<SegmentWithDescriptor> dataSegmentIterator(
 
           return new SegmentWithDescriptor(
               dataSegmentProvider.fetchSegment(segmentId, channelCounters, 
isReindex),
+              
loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, 
channelCounters),

Review Comment:
   Shouldn't this be null for segment'sWithoutLocation?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> realtimeAndHistoricalSegments;
+
+      // Fetch the realtime segments first, so that we don't miss any segment 
if they get handed off between the two

Review Comment:
   ```suggestion
         // Fetch the realtime segments and segments loaded on the historical 
first, so that we don't miss any segment if they get handed off between the two
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java:
##########
@@ -76,6 +80,13 @@ public DataSegmentProvider dataSegmentProvider()
     return dataSegmentProvider;
   }
 
+  @Override

Review Comment:
   This can be null as well. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java:
##########
@@ -65,6 +82,18 @@ public ResourceHolder<Segment> getOrLoad()
     return segmentSupplier.get();
   }
 
+  public <QueryType, RowType> 
Pair<LoadedSegmentDataProvider.DataServerQueryStatus, Yielder<RowType>> 
fetchRowsFromDataServer(
+      Query<QueryType> query,
+      Function<Sequence<QueryType>, Sequence<RowType>> mappingFunction,
+      Closer closer
+  ) throws IOException
+  {
+    if (loadedSegmentDataProvider == null) {
+      throw new RE("loadedSegmentDataProvider was null. Fetching segments from 
servers is not supported for segment[%s]", descriptor);

Review Comment:
   Lets throw DruidException.defensive here ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =
           
FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource,
 intervals), true);
 
-      if (dataSegments.isEmpty()) {
+      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<DataSegment> unifiedSegmentView = new 
HashSet<>(publishedUsedSegments);
+      for (ImmutableSegmentLoadInfo segmentLoadInfo : 
realtimeAndHistoricalSegments) {
+        ImmutableSet<DruidServerMetadata> 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<DruidServerMetadata> realtimeServerMetadata
+            = servers.stream()
+                     .filter(druidServerMetadata -> 
includeSegmentSource.getUsedServerTypes()
+                                                                        
.contains(druidServerMetadata.getType())
+                     )
+                     .collect(Collectors.toSet());
+        if (!realtimeServerMetadata.isEmpty()) {
+          realtimeCount += 1;
+          DataSegmentWithLocation dataSegmentWithLocation = new 
DataSegmentWithLocation(
+              segmentLoadInfo.getSegment(),
+              realtimeServerMetadata
+          );
+          unifiedSegmentView.add(dataSegmentWithLocation);
+        }

Review Comment:
   Nit: can you mention the else part and why is it a no op. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java:
##########
@@ -1163,14 +1167,67 @@ private QueryKit makeQueryControllerToolKit()
 
   private DataSegmentTimelineView makeDataSegmentTimelineView()
   {
+    final SegmentSource includeSegmentSource = 
MultiStageQueryContext.getSegmentSources(
+        task.getQuerySpec()
+            .getQuery()
+            .context()
+    );
+
+    final boolean includeRealtime = 
SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
     return (dataSource, intervals) -> {
-      final Collection<DataSegment> dataSegments =
+      final Iterable<ImmutableSegmentLoadInfo> 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.
+      if (includeRealtime) {
+        realtimeAndHistoricalSegments = 
context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+      } else {
+        realtimeAndHistoricalSegments = ImmutableList.of();
+      }
+
+      // Fetch all published used segments from the metadata store.
+      final Collection<DataSegment> publishedUsedSegments =

Review Comment:
   Can be done in a follow up PR. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.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.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.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;
+import org.apache.druid.rpc.RpcException;
+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 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;
+  private final QueryToolChestWarehouse warehouse;
+
+  public LoadedSegmentDataProvider(
+      String dataSource,
+      ChannelCounters channelCounters,
+      ServiceClientFactory serviceClientFactory,
+      CoordinatorClient coordinatorClient,
+      ObjectMapper objectMapper,
+      QueryToolChestWarehouse warehouse
+  )
+  {
+    this.dataSource = dataSource;
+    this.channelCounters = channelCounters;
+    this.serviceClientFactory = serviceClientFactory;
+    this.coordinatorClient = coordinatorClient;
+    this.objectMapper = objectMapper;
+    this.warehouse = warehouse;
+  }
+
+  @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.
+   * - 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 <QueryType> result return type for the query from the data server
+   * @param <RowType> type of the result rows after parsing from QueryType 
object
+   */
+  public <RowType, QueryType> Pair<DataServerQueryStatus, Yielder<RowType>> 
fetchRowsFromDataServer(
+      Query<QueryType> query,
+      RichSegmentDescriptor segmentDescriptor,
+      Function<Sequence<QueryType>, Sequence<RowType>> mappingFunction,
+      Closer closer
+  ) throws IOException
+  {
+    final Query<QueryType> preparedQuery = Queries.withSpecificSegments(
+        query.withDataSource(new TableDataSource(dataSource)),
+        ImmutableList.of(segmentDescriptor)
+    );
+
+    Set<DruidServerMetadata> servers = segmentDescriptor.getServers();
+    DataServerClient dataServerClient = 
makeDataServerClient(FixedSetServiceLocator.forDruidServerMetadata(servers));

Review Comment:
   Lets mark these final since they are used in the lambda



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to