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


##########
server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.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;
+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<AppendableByteArrayInputStream, InputStream>

Review Comment:
   Can you add some javadocs which are implementation specific and how it ts 
different from directDruidClient response handler 



##########
server/src/main/java/org/apache/druid/discovery/DataServerClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 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;
+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.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 final ServiceClient serviceClient;
+  private final ObjectMapper objectMapper;
+  private final ScheduledExecutorService queryCancellationExecutor;
+
+  public DataServerClient(
+      ServiceClientFactory serviceClientFactory,
+      FixedSetServiceLocator fixedSetServiceLocator,
+      ObjectMapper objectMapper
+  )
+  {
+    this.serviceClient = serviceClientFactory.makeClient(
+        NodeRole.INDEXER.getJsonName(),
+        fixedSetServiceLocator,
+        StandardRetryPolicy.noRetries()
+    );
+    this.objectMapper = objectMapper;
+    this.queryCancellationExecutor = 
Execs.scheduledSingleThreaded("query-cancellation-executor");
+  }
+
+  public <T> Sequence<T> run(Query<T> query, ResponseContext responseContext, 
JavaType queryResultType, Closer closer)
+  {
+    final String basePath = "/druid/v2/";

Review Comment:
   We can make this part a class variable. 



##########
server/src/main/java/org/apache/druid/rpc/ServiceLocation.java:
##########
@@ -47,6 +53,42 @@ 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.getHost());
+    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) {

Review Comment:
   I think host can never be null. Maybe a precondition check is required



##########
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

Review Comment:
   Please add a test for empty and null locations. 



##########
server/src/main/java/org/apache/druid/discovery/DataServerClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 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;
+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.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 final ServiceClient serviceClient;
+  private final ObjectMapper objectMapper;
+  private final ScheduledExecutorService queryCancellationExecutor;
+
+  public DataServerClient(
+      ServiceClientFactory serviceClientFactory,
+      FixedSetServiceLocator fixedSetServiceLocator,
+      ObjectMapper objectMapper
+  )
+  {
+    this.serviceClient = serviceClientFactory.makeClient(
+        NodeRole.INDEXER.getJsonName(),
+        fixedSetServiceLocator,
+        StandardRetryPolicy.noRetries()
+    );
+    this.objectMapper = objectMapper;
+    this.queryCancellationExecutor = 
Execs.scheduledSingleThreaded("query-cancellation-executor");

Review Comment:
   Can we make a pool for this. For each segment you are creating a new 
executor service which seems wasteful. 



##########
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));
+    QueryToolChest<QueryType, Query<QueryType>> toolChest = 
warehouse.getToolChest(query);
+    Function<QueryType, QueryType> preComputeManipulatorFn =
+        toolChest.makePreComputeManipulatorFn(query, 
MetricManipulatorFns.deserializing());
+
+    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);
+    final ResponseContext responseContext = new DefaultResponseContext();
+
+    Pair<DataServerQueryStatus, Yielder<RowType>> statusSequencePair;
+    try {
+      statusSequencePair = RetryUtils.retry(
+          () -> {
+            Sequence<QueryType> sequence = dataServerClient.run(preparedQuery, 
responseContext, queryResultType, closer)

Review Comment:
   What happen's if sequence is interrupted or closed?



##########
server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.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;
+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<AppendableByteArrayInputStream, InputStream>
+{
+  private static final Logger log = new 
Logger(DataServerResponseHandler.class);
+  private final String queryId;
+  private final ResponseContext responseContext;
+  private final ObjectMapper objectMapper;
+
+  public <T> DataServerResponseHandler(Query<T> query, ResponseContext 
responseContext, ObjectMapper objectMapper)
+  {
+    this.queryId = query.getId();
+    this.responseContext = responseContext;
+    this.objectMapper = objectMapper;
+  }
+
+  @Override
+  public ClientResponse<AppendableByteArrayInputStream> 
handleResponse(HttpResponse response, TrafficCop trafficCop)
+  {
+    log.debug("Received response status[%s] for queryId[%s]", 
response.getStatus(), queryId);

Review Comment:
   What about metrics like bytes scanned and stuff ?



##########
server/src/main/java/org/apache/druid/discovery/DataServerClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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 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;
+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.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 final ServiceClient serviceClient;
+  private final ObjectMapper objectMapper;
+  private final ScheduledExecutorService queryCancellationExecutor;
+
+  public DataServerClient(
+      ServiceClientFactory serviceClientFactory,
+      FixedSetServiceLocator fixedSetServiceLocator,
+      ObjectMapper objectMapper
+  )
+  {
+    this.serviceClient = serviceClientFactory.makeClient(
+        NodeRole.INDEXER.getJsonName(),
+        fixedSetServiceLocator,
+        StandardRetryPolicy.noRetries()
+    );
+    this.objectMapper = objectMapper;
+    this.queryCancellationExecutor = 
Execs.scheduledSingleThreaded("query-cancellation-executor");

Review Comment:
   Who closes this executor service. 



##########
server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.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;
+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<AppendableByteArrayInputStream, InputStream>
+{
+  private static final Logger log = new 
Logger(DataServerResponseHandler.class);
+  private final String queryId;
+  private final ResponseContext responseContext;
+  private final ObjectMapper objectMapper;
+
+  public <T> DataServerResponseHandler(Query<T> query, ResponseContext 
responseContext, ObjectMapper objectMapper)
+  {
+    this.queryId = query.getId();
+    this.responseContext = responseContext;
+    this.objectMapper = objectMapper;
+  }
+
+  @Override
+  public ClientResponse<AppendableByteArrayInputStream> 
handleResponse(HttpResponse response, TrafficCop trafficCop)
+  {
+    log.debug("Received response status[%s] for queryId[%s]", 
response.getStatus(), queryId);

Review Comment:
   Should we check for timeout?
   Shoudnt traffic cop be used for backpressure?
   We can do these things in a follow up PR as well. 



##########
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));
+    QueryToolChest<QueryType, Query<QueryType>> toolChest = 
warehouse.getToolChest(query);
+    Function<QueryType, QueryType> preComputeManipulatorFn =
+        toolChest.makePreComputeManipulatorFn(query, 
MetricManipulatorFns.deserializing());
+
+    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);
+    final ResponseContext responseContext = new DefaultResponseContext();
+
+    Pair<DataServerQueryStatus, Yielder<RowType>> statusSequencePair;
+    try {
+      statusSequencePair = RetryUtils.retry(
+          () -> {
+            Sequence<QueryType> sequence = dataServerClient.run(preparedQuery, 
responseContext, queryResultType, closer)
+                                                           
.map(preComputeManipulatorFn);
+            final List<SegmentDescriptor> 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<RowType> 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 IOE(
+                    "Segment[%s] could not be found on data server, but 
segment was not handed off.",
+                    segmentDescriptor
+                );
+              }
+            }
+          },
+          throwable -> !(throwable instanceof QueryInterruptedException && 
throwable.getCause() instanceof InterruptedException),
+          numRetriesOnMissingSegments
+      );
+
+      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) {
+      Throwables.propagateIfPossible(e, IOE.class);
+      throw new IOE(e, "Exception while fetching rows for query from 
dataservers[%s]", servers);
+    }
+  }
+
+  private static List<SegmentDescriptor> getMissingSegments(final 
ResponseContext responseContext)
+  {
+    List<SegmentDescriptor> missingSegments = 
responseContext.getMissingSegments();
+    if (missingSegments == null) {
+      return ImmutableList.of();
+    }
+    return missingSegments;
+  }
+
+  private static boolean checkSegmentHandoff(
+      CoordinatorClient coordinatorClient,
+      String dataSource,
+      SegmentDescriptor segmentDescriptor
+  ) throws IOE
+  {
+    Boolean wasHandedOff;
+    try {
+      wasHandedOff = RetryUtils.retry(
+          () -> 
FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, 
segmentDescriptor), true),

Review Comment:
   This retry is not required since logic is built inside service client. 



##########
server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java:
##########
@@ -42,7 +43,7 @@ public class DruidServerMetadata
   @JsonCreator
   public DruidServerMetadata(
       @JsonProperty("name") String name,
-      @JsonProperty("host") String hostAndPort,
+      @JsonProperty("host") @Nullable String hostAndPort,

Review Comment:
   Can you add a comment that either hostAndPort is set or hostAndTlsPort is 
set 



##########
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));
+    QueryToolChest<QueryType, Query<QueryType>> toolChest = 
warehouse.getToolChest(query);
+    Function<QueryType, QueryType> preComputeManipulatorFn =
+        toolChest.makePreComputeManipulatorFn(query, 
MetricManipulatorFns.deserializing());
+
+    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);
+    final ResponseContext responseContext = new DefaultResponseContext();
+
+    Pair<DataServerQueryStatus, Yielder<RowType>> statusSequencePair;
+    try {
+      statusSequencePair = RetryUtils.retry(

Review Comment:
   Please document why and external retry logic is required since we are using 
the service client already. 



##########
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));
+    QueryToolChest<QueryType, Query<QueryType>> toolChest = 
warehouse.getToolChest(query);
+    Function<QueryType, QueryType> preComputeManipulatorFn =
+        toolChest.makePreComputeManipulatorFn(query, 
MetricManipulatorFns.deserializing());
+
+    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);
+    final ResponseContext responseContext = new DefaultResponseContext();
+
+    Pair<DataServerQueryStatus, Yielder<RowType>> statusSequencePair;
+    try {
+      statusSequencePair = RetryUtils.retry(
+          () -> {
+            Sequence<QueryType> sequence = dataServerClient.run(preparedQuery, 
responseContext, queryResultType, closer)
+                                                           
.map(preComputeManipulatorFn);
+            final List<SegmentDescriptor> 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<RowType> 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);

Review Comment:
   can we log the counts in the stage somewhere ?
   



-- 
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