You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/06/26 05:35:46 UTC

[GitHub] [druid] jihoonson opened a new pull request #10082: Fix RetryQueryRunner to actually do the job

jihoonson opened a new pull request #10082:
URL: https://github.com/apache/druid/pull/10082


   ### Description
   
   `RetryQueryRunner` is responsible for retrying the query when some segments are missing during the query (it's possible since the coordinator can move segments anytime). However, it currently doesn't work as expected since it checks the missing segments in the response context _before_ issuing the query to the query nodes. This PR fixes this bug and adds a sanity check that makes checking missing segments failed if the broker hasn't gotten all responses from the query nodes yet.
   
   Some integration tests will be added as a follow-up.
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/licenses.yaml)
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448229976



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       Added a flag and made the historicals fail when response context is truncated.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-652572586


   @gianm thanks for the review. As Travis is extremely slow recently, I'd like to address your last comments in a follow-up with doc for the known issue with response context not shared by subqueries.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447174398



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       I'm not sure I agree that extending a production class in test code is cleaner than adding a test-only option to production code. They're equivalent in terms of test authenticity, and I'm biased towards thinking that extending things is bad in general (because it splits interconnected logic over two different files). So the test-only option sounds good to me.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446450845



##########
File path: processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
##########
@@ -112,6 +112,16 @@
         "uncoveredIntervalsOverflowed",
             (oldValue, newValue) -> (boolean) oldValue || (boolean) newValue
     ),
+    /**
+     * Expected remaining number of responses from query nodes.
+     * The value is initialized in {@code CachingClusteredClient} when it initializes the connection to the query nodes,
+     * and is updated whenever they respond (@code DirectDruidClient). {@code RetryQueryRunner} uses this value to
+     * check if the {@link #MISSING_SEGMENTS} is valid.
+     */
+    REMAINING_RESPONSES_FROM_QUERY_NODES(

Review comment:
       👍 added.

##########
File path: server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
##########
@@ -618,6 +619,7 @@ private void addSequencesFromServer(
         final SortedMap<DruidServer, List<SegmentDescriptor>> segmentsByServer
     )
     {
+      responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_NODES, segmentsByServer.size());

Review comment:
       I think the response contexts are shared between subqueries, but it won't be a problem because subqueries are not executed in parallel for now. Same for `UnionDataSource`. However, I agree that it should be per subquery for the future. I made needed changes to set subqueryId for union and store number of servers remaining per subquery or query.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447336897



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       > Is deferring `baseRunner.run()` going to have any unintended consequences? For example, is it going to meaningfully delay initiation of connections to historicals? I'm not sure, but in general this area of the system is sensitive to when exactly things get called, so it would be good to look into this when changing it.
   
   Good question. It does defer initiating connections to query servers if the broker hasn't initiated them yet. AFAIT, there don't seem obvious unintended side effects. [All these query runners running on top of `RetryQueryRunner` are not aware of that the query will be run remotely](https://github.com/apache/druid/blob/master/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java#L393-L425).

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       I agree with @gianm. I'm more worried about that the tests may be able to test different code than production if we forget to sync when we modify this part later in the future.

##########
File path: processing/src/main/java/org/apache/druid/query/Query.java
##########
@@ -154,6 +161,17 @@ default String getSqlQueryId()
     return null;
   }
 
+  /**
+   * Returns a most relevant ID of this query; if it is a subquery, this will return its subquery ID.
+   * If it is a regular query without subqueries, this will return its query ID.
+   * This method should be called after the relevant ID is assigned using {@link #withId} or {@link #withSubQueryId}.
+   */
+  default String getMostRelevantId()

Review comment:
       Renamed as suggested and added a debug log.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446480325



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Oh, I actually found the response context is not shared between subqueries when inlining them. It is shared in UnionQueryRunner though. I think it makes sense to not share the context when inlining subqueries.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446626252



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       Hmm, I feel like this could be done without polluting the production code with test stuff. 
   
   How about:
   * making `RetryingSequenceIterator` visible for testing so you can extend and override `hasNext` to do this runnable thing
   * [move building the retry sequence in the run method](https://github.com/apache/druid/pull/10082/files#diff-661e59db1678430f72450fd2fa5ce061R107) into a new method that you can override, so to have it make the test iterator with the runnable instead

##########
File path: server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
##########
@@ -497,10 +499,11 @@ public void testTimeseriesCaching()
     );
 
 
-    TimeseriesQuery query = builder.intervals("2011-01-01/2011-01-10")
-                                   .aggregators(RENAMED_AGGS)
-                                   .postAggregators(RENAMED_POST_AGGS)
-                                   .build();
+    TimeseriesQuery query = (TimeseriesQuery) builder.intervals("2011-01-01/2011-01-10")
+                                                     .aggregators(RENAMED_AGGS)
+                                                     .postAggregators(RENAMED_POST_AGGS)
+                                                     .build()
+                                                     .withId("queryId");

Review comment:
       I'm surprised that the builder doesn't have an option to set queryId.

##########
File path: processing/src/main/java/org/apache/druid/query/Query.java
##########
@@ -154,6 +161,17 @@ default String getSqlQueryId()
     return null;
   }
 
+  /**
+   * Returns a most relevant ID of this query; if it is a subquery, this will return its subquery ID.
+   * If it is a regular query without subqueries, this will return its query ID.
+   * This method should be called after the relevant ID is assigned using {@link #withId} or {@link #withSubQueryId}.
+   */
+  default String getMostRelevantId()

Review comment:
       nit: I think `getMostSpecificId()` might be a better name for this method
   
   tangent: it might be nice if the debug logs in `DirectDruidClient` logged both queryId and subQueryId if set




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-652569145


   Btw, if you can do the above two things (and add integration tests) in follow-ups before the next release, I think that would be OK. Let us know what you prefer @jihoonson.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446451764



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest

Review comment:
       I agree the integration tests will be more useful and am actually working on it as commented in the PR description. It will be added in a follow-up PR. BTW, I think the idea described in https://github.com/apache/druid/pull/10082#discussion_r446451237 seems useful for integration tests. That way, we can emulate the what can really happen in real clusters.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448229976



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       Added a flag and made the historicals fail when response context is truncated. Manually tested this new behavior.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-651465790


   BTW, I'm not sure whether or not our tests test `CachingClusteredClient` in the following cases, so tested them manually.
   
   - With/without broker cache
   - With/without result-level cache
   - With/without queries with subqueries
   - Unknown datasource


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446480359



##########
File path: server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
##########
@@ -618,6 +619,7 @@ private void addSequencesFromServer(
         final SortedMap<DruidServer, List<SegmentDescriptor>> segmentsByServer
     )
     {
+      responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_NODES, segmentsByServer.size());

Review comment:
       Oh, I actually found the response context is not shared between subqueries when inlining them. It is shared in UnionQueryRunner though. I think it makes sense to not share the context when inlining subqueries.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446451237



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Before the last commit, `baseRunner.run()` was executed when `RetryQueryRunner.run()` is called. Since `baseRunner` is the query runner created by `CachingClusteredClient`, `RetryQueryRunner.run()` created the query distribution tree immediately. As a result, I could make the tests more deterministic by emulating moving segments between `sequence = RetryQueryRunner.run()` and `sequence.toList()`.
   
   In the last commit, I changed the code to defer calling `baseRunner.run()` until it has to get executed. This will minimize the chance of segments to move between the time to create query distribution tree and the time to actually send the query to servers. To keep the tests deterministic, I added a new constructor parameter, `runnableAfterFirstAttempt`, which is executed after creating query distribution true for the first attempt. This new parameter must be no-op in production code, but will be used only for testing. I think this parameter will be also useful for integration tests; I can add a new coordinator API to trigger segment balancing and call it after the first attempt using `runnableAfterFirstAttempt`.

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  private int numTotalRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+  }
+
+  @VisibleForTesting
+  int getNumTotalRetries()
+  {
+    return numTotalRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    final Sequence<T> baseSequence = baseRunner.run(queryPlus, context);
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseSequence);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                numTotalRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final int remainingResponses = Preconditions.checkNotNull(
+        (Integer) context.get(Key.REMAINING_RESPONSES_FROM_QUERY_NODES),
+        "%s in responseContext",
+        Key.REMAINING_RESPONSES_FROM_QUERY_NODES.getName()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence given in the constructor. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * nodes (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private Sequence<T> sequence;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(QueryPlus<T> queryPlus, ResponseContext context, Sequence<T> baseSequence)
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.sequence = baseSequence;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (sequence != null) {
+        return true;
+      } else {
+        final List<SegmentDescriptor> missingSegments = getMissingSegments(context);
+        if (missingSegments.isEmpty()) {
+          return false;
+        } else if (retryCount >= config.getNumTries()) {
+          if (!config.isReturnPartialResults()) {
+            throw new SegmentMissingException("No results found for segments[%s]", missingSegments);
+          } else {
+            return false;
+          }
+        } else {
+          LOG.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), retryCount++);

Review comment:
       Oops, just copied it from the original code. Split incrementing count from the logging.

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  private int numTotalRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+  }
+
+  @VisibleForTesting
+  int getNumTotalRetries()

Review comment:
       Renamed.

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);
+    Assert.assertEquals(expectedTimeseriesResult(queryResult.size()), queryResult);
+  }
+
+  @Test
+  public void testRetryUntilWeGetFullResult()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(100, false), // retry up to 100
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertTrue(0 < queryRunner.getNumTotalRetries());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testFailWithPartialResultsAfterRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    dropSegmentFromServer(servers.get(0));
+
+    expectedException.expect(SegmentMissingException.class);
+    expectedException.expectMessage("No results found for segments");
+    try {
+      sequence.toList();
+    }
+    finally {
+      Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    }
+  }
+
+  private void prepareCluster(int numServers)
+  {
+    for (int i = 0; i < numServers; i++) {
+      final DataSegment segment = newSegment(SCHEMA_INFO.getDataInterval(), i);
+      addServer(
+          SimpleServerView.createServer(i + 1),
+          segment,
+          segmentGenerator.generate(segment, SCHEMA_INFO, Granularities.NONE, 10)
+      );
+    }
+  }
+
+  private Pair<SegmentId, QueryableIndex> dropSegmentFromServer(DruidServer fromServer)

Review comment:
       Added for both methods.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448135388



##########
File path: processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
##########
@@ -112,6 +114,30 @@
         "uncoveredIntervalsOverflowed",
             (oldValue, newValue) -> (boolean) oldValue || (boolean) newValue
     ),
+    /**
+     * Map of most relevant query ID to remaining number of responses from query nodes.
+     * The value is initialized in {@code CachingClusteredClient} when it initializes the connection to the query nodes,
+     * and is updated whenever they respond (@code DirectDruidClient). {@code RetryQueryRunner} uses this value to
+     * check if the {@link #MISSING_SEGMENTS} is valid.
+     *
+     * Currently, the broker doesn't run subqueries in parallel, the remaining number of responses will be updated
+     * one by one per subquery. However, since we are planning to parallelize running subqueries, we store them

Review comment:
       > I'm not sure if we are, but, we also haven't decided definitely _not_ to.
   
   Heh, rephrased to sound more chill.
   
   > I think it would also be okay to not handle parallel queries here, but instead build in a sanity check that verifies the subqueries are issued in series. Maybe by verifying that when the id changes, the previous number of responses remaining must be down to zero. It's up to you. I think your current code is okay too.
   
   Hmm, how could it be used? Is it to make sure that the subqueries will be issued in series until we parallelize running them?

##########
File path: server/src/main/java/org/apache/druid/client/DirectDruidClient.java
##########
@@ -231,7 +237,19 @@ private InputStream dequeue() throws InterruptedException
 
           final boolean continueReading;
           try {
+            if (query.getId() != null && query.getSubQueryId() != null) {

Review comment:
       Changed to always print trace-level log.

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostSpecificId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostSpecificId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);

Review comment:
       Oops thanks. For the comment, I wrote some comments [here](https://github.com/apache/druid/pull/10082/files#diff-661e59db1678430f72450fd2fa5ce061R133). Do you think it's not clear?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448109639



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       Ah, this is a really good point, I agree it should be an option to be an exception if the list is truncated. 
   
   I'm not sure what better we can really do without a fairly large refactoring involving making the broker keep track of which segments went to which direct druid client/sequence, and allowing historicals to basically give up if missing so many that the list would be truncated, returning an empty sequence and an indicator that **all** of the segments it was queried for should be retried.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448136240



##########
File path: server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java
##########
@@ -329,15 +329,18 @@ private DataSource inlineIfNecessary(
         }
       } else if (canRunQueryUsingLocalWalker(subQuery) || canRunQueryUsingClusterWalker(subQuery)) {
         // Subquery needs to be inlined. Assign it a subquery id and run it.
-        final Query subQueryWithId = subQuery.withSubQueryId(UUID.randomUUID().toString());
+        final Query subQueryWithId = subQuery.withDefaultSubQueryId();
 
         final Sequence<?> queryResults;
 
         if (dryRun) {
           queryResults = Sequences.empty();
         } else {
           final QueryRunner subqueryRunner = subQueryWithId.getRunner(this);
-          queryResults = subqueryRunner.run(QueryPlus.wrap(subQueryWithId));
+          queryResults = subqueryRunner.run(
+              QueryPlus.wrap(subQueryWithId),
+              DirectDruidClient.makeResponseContextForQuery()

Review comment:
       > Now that I think about it, though, making a new context here will mean we aren't going to properly return context from subqueries up to the original caller. This includes not reporting missing segments in the case where `RetryQueryRunnerConfig.isReturnPartialResults = true`.
   
   Thanks. Looking at the keys in responseContext, it makes sense to share it between subqueries (at least for some of them such as `cpuConsumed`) and seems doable. But I would like to do it in a separate PR since this PR is already big. I will document it as a known issue.
   
   BTW, the responseContext seems to be used for diverse purposes now. Some of them are:
   
   - To inform something important to users after query is done, such as `uncoveredIntervals`. 
   - To store some intermediate state in the broker during query processing, such as `remainingResponsesFromQueryServers`.
   - To gather some metrics from query servers such as `cpuConsumed`.
   - A part of the cache validation mechanism using `ETag`.
   
   I think it would probably be better split those into the first one and others at some point.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis merged pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
clintropolis merged pull request #10082:
URL: https://github.com/apache/druid/pull/10082


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-652576873


   > @gianm thanks for the review. As Travis is extremely slow recently, I'd like to address your last comments in a follow-up with doc for the known issue with response context not shared by subqueries.
   
   That works for me. I think the follow-ups we'd want before the next release would be:
   
   1. Add integration tests.
   2. Add the new error code to the documentation: https://github.com/apache/druid/pull/10082#discussion_r448525940
   3. Modify this method name: https://github.com/apache/druid/pull/10082#discussion_r448526783
   
   For documenting the limitations of response context, I'm actually not sure what we should do, because I realized that currently they aren't documented at all. They are a secret feature, I suppose. So I don't think there's any need to document their limitations if we aren't including it as a documented feature in the first place. I think there are two good options here:
   
   - Do nothing, don't add any docs.
   - Document response contexts as an 'alpha' or 'beta' level feature, and include documentation about their limitations (they can be too long and will either truncate or fail the query; and they aren't currently collected from all subqueries).


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446335802



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  private int numTotalRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+  }
+
+  @VisibleForTesting
+  int getNumTotalRetries()

Review comment:
       Should be `getTotalNumRetries()`. "Num total retries" makes it sound like the number of times we did a total retry.

##########
File path: processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
##########
@@ -112,6 +112,16 @@
         "uncoveredIntervalsOverflowed",
             (oldValue, newValue) -> (boolean) oldValue || (boolean) newValue
     ),
+    /**
+     * Expected remaining number of responses from query nodes.
+     * The value is initialized in {@code CachingClusteredClient} when it initializes the connection to the query nodes,
+     * and is updated whenever they respond (@code DirectDruidClient). {@code RetryQueryRunner} uses this value to
+     * check if the {@link #MISSING_SEGMENTS} is valid.
+     */
+    REMAINING_RESPONSES_FROM_QUERY_NODES(

Review comment:
       This should be added to `DirectDruidClient.removeMagicResponseContextFields` or something similar, so it doesn't end up in the response to the user.

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest

Review comment:
       A general comment on testing: we should have a real integration test for this. It is the kind of thing that is tough to get right in unit tests, as evidence by the fact that we already had an existing RetryQueryRunnerTest, but it isn't testing the right thing.
   
   I reviewed the unit test and it seems reasonable enough (with some comments), but for this particular functionality we need an integration test that uses the real servers and a real network.

##########
File path: server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
##########
@@ -618,6 +619,7 @@ private void addSequencesFromServer(
         final SortedMap<DruidServer, List<SegmentDescriptor>> segmentsByServer
     )
     {
+      responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_NODES, segmentsByServer.size());

Review comment:
       I'm not sure this will work properly if there are multiple CCC queries associated with a given Druid query. This can happen with union queries and with subqueries. I believe they share response contexts, and this `put` would potentially cause things to get clobbered. Could you please look into this?
   
   Btw, the reason I believe they share contexts is that I only currently see concurrent contexts getting created in QueryLifecycle (which can wrap multiple CCC queries).
   
   If it is a real issue, we might be able to address it by splitting out the number of servers remaining by subquery ID. However, in that case, we need to make sure subquery ID is set for union datasources. I don't think it is currently.

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Is there a guarantee that there will be 1 retry? Why won't the the correct, new server for the segment be selected the first time?

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  private int numTotalRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+  }
+
+  @VisibleForTesting
+  int getNumTotalRetries()
+  {
+    return numTotalRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    final Sequence<T> baseSequence = baseRunner.run(queryPlus, context);
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseSequence);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                numTotalRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final int remainingResponses = Preconditions.checkNotNull(
+        (Integer) context.get(Key.REMAINING_RESPONSES_FROM_QUERY_NODES),
+        "%s in responseContext",
+        Key.REMAINING_RESPONSES_FROM_QUERY_NODES.getName()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence given in the constructor. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * nodes (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private Sequence<T> sequence;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(QueryPlus<T> queryPlus, ResponseContext context, Sequence<T> baseSequence)
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.sequence = baseSequence;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (sequence != null) {
+        return true;
+      } else {
+        final List<SegmentDescriptor> missingSegments = getMissingSegments(context);
+        if (missingSegments.isEmpty()) {
+          return false;
+        } else if (retryCount >= config.getNumTries()) {
+          if (!config.isReturnPartialResults()) {
+            throw new SegmentMissingException("No results found for segments[%s]", missingSegments);
+          } else {
+            return false;
+          }
+        } else {
+          LOG.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), retryCount++);

Review comment:
       Should be `++retryCount`, right? (You would want the first retry to say "Retry attempt [1]".)
   
   However, prefix and postfix incrementing in log messages can make things harder to read, so IMO it'd be better to split this up into two separate lines.

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -766,7 +766,6 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
                    .withDruidCluster(cluster)
                    .withLoadManagementPeons(loadManagementPeons)
                    .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withBalancerReferenceTimestamp(DateTimes.nowUtc())

Review comment:
       Is this change related?

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);
+    Assert.assertEquals(expectedTimeseriesResult(queryResult.size()), queryResult);
+  }
+
+  @Test
+  public void testRetryUntilWeGetFullResult()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(100, false), // retry up to 100
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertTrue(0 < queryRunner.getNumTotalRetries());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testFailWithPartialResultsAfterRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    dropSegmentFromServer(servers.get(0));
+
+    expectedException.expect(SegmentMissingException.class);
+    expectedException.expectMessage("No results found for segments");
+    try {
+      sequence.toList();
+    }
+    finally {
+      Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    }
+  }
+
+  private void prepareCluster(int numServers)
+  {
+    for (int i = 0; i < numServers; i++) {
+      final DataSegment segment = newSegment(SCHEMA_INFO.getDataInterval(), i);
+      addServer(
+          SimpleServerView.createServer(i + 1),
+          segment,
+          segmentGenerator.generate(segment, SCHEMA_INFO, Granularities.NONE, 10)
+      );
+    }
+  }
+
+  private Pair<SegmentId, QueryableIndex> dropSegmentFromServer(DruidServer fromServer)

Review comment:
       Could you add some javadocs to these methods explaining what they do?
   
   It's non-obvious that `dropSegmentFromServer` doesn't modify the server view, and that `dropSegmentFromServerAndAddNewServerForSegment` _does_ modify the server view, but only for the new server, not the old one.
   
   Anywhere there is a lack of symmetry and obviousness like this, doc comments are especially important.

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);
+    Assert.assertEquals(expectedTimeseriesResult(queryResult.size()), queryResult);
+  }
+
+  @Test
+  public void testRetryUntilWeGetFullResult()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(100, false), // retry up to 100
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertTrue(0 < queryRunner.getNumTotalRetries());

Review comment:
       Is there a guarantee that there will be more than zero retries? Why won't the the correct, new server for the segment be selected the first time?

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);

Review comment:
       Could we use exact sizes here? (If we can't be sure what we'll get, maybe check that there's one of two sizes, and run the test repeatedly for a minimum number of times and verify that we actually get both.)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447165811



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -766,7 +766,6 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
                    .withDruidCluster(cluster)
                    .withLoadManagementPeons(loadManagementPeons)
                    .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withBalancerReferenceTimestamp(DateTimes.nowUtc())

Review comment:
       True, and I think it's fine if you want to leave this fix here, but next time it would be cleaner to do two separate patches.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447991275



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       Ah, I don't think it makes it overly complex or anything, just more complex than if it wasn't there, so more ugly like you said. I don't think this necessarily even needs to change, I'm just being a hater 😛 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-652645612


   @gianm @clintropolis thank you for the review. I'll address @gianm's last comments in follow-ups. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447167924



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Oh, I see. It's because you called `run` before `dropSegmentFromServerAndAddNewServerForSegment`.
   
   Is deferring `baseRunner.run()` going to have any unintended consequences? For example, is it going to meaningfully delay initiation of connections to historicals? I'm not sure, but in general this area of the system is sensitive to when exactly things get called, so it would be good to look into this when changing it.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #10082:
URL: https://github.com/apache/druid/pull/10082#issuecomment-652097170


   The test coverage check now complains about these classes:
   
   ```
   ------------------------------------------------------------------------------
   |     lines      |    branches    |   functions    |   path
   ------------------------------------------------------------------------------
   |  33% (1/3)     |   0% (0/2)     |  83% (10/12)   | org/apache/druid/query/Query.java
   | 100% (3/3)     | 100% (0/0)     | 100% (6/6)     | org/apache/druid/query/UnionQueryRunner.java
   |  46% (19/41)   |  36% (8/22)    |  61% (19/31)   | org/apache/druid/query/Druids.java
   | 100% (6/6)     | 100% (2/2)     | 100% (3/3)     | org/apache/druid/query/context/ResponseContext.java
   |  16% (1/6)     |   0% (0/2)     |  70% (7/10)    | org/apache/druid/query/topn/TopNQueryBuilder.java
   |  33% (2/6)     |   0% (0/2)     |  33% (1/3)     | org/apache/druid/query/groupby/GroupByQuery.java
   ------------------------------------------------------------------------------
   Total diff coverage:
    - lines: 49% (32/65)
    - branches: 33% (10/30)
    - functions: 70% (46/65)
   ERROR: Insufficient line coverage of 49% (32/65). Required 50%.
   ERROR: Insufficient branch coverage of 33% (10/30). Required 50%.
   ```
   
   `Druids`, `TopNQueryBuilder`, and `GroupByQuery.Builder` are for query builders which are used only in unit tests, so I suppressed the check for them. For `Query`, it's about the new default `getMostSpecificId` method which I don't expect any class will override it unless some query type has its own ID system. I think we can ignore it.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448136101



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       Hmm, I was aware of this issue, but ignored since the max size of the serialized responseContext is 7k. The missingSegments is a list of `SegmentDescriptor` which has an `interval`, a `version`, and a `partitionNumber`. Since its serialized size is about 80 bytes, each historical can report up to about 90 missing segments. I thought it wouldn't happen often, but well, I guess it depends on how your cluster is configured and how many segments the query reads.
   
   > I'm not sure what better we can really do without a fairly large refactoring involving making the broker keep track of which segments went to which direct druid client/sequence, and allowing historicals to basically give up if missing so many that the list would be truncated, returning an empty sequence and an indicator that **all** of the segments it was queried for should be retried.
   
   Hmm, I was thinking that we don't have to keep track of all segments, but can send a flag in the query context to historicals which makes the query failed immediately if the serialized responseContext is truncated. The broker will catch the failure and cancel the entire query. This won't retry on the truncated serialization, but I think it makes sense since it's a sort of failure of the retry system. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448525940



##########
File path: processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java
##########
@@ -105,6 +106,8 @@ private static String getErrorCodeFromThrowable(Throwable e)
       return RESOURCE_LIMIT_EXCEEDED;
     } else if (e instanceof UnsupportedOperationException) {
       return UNSUPPORTED_OPERATION;
+    } else if (e instanceof TruncatedResponseContextException) {
+      return TRUNCATED_RESPONSE_CONTEXT;

Review comment:
       This should be added to the documentation (all of these error codes are spelled out in a table in `querying/querying.md`).

##########
File path: processing/src/main/java/org/apache/druid/query/QueryContexts.java
##########
@@ -344,6 +346,19 @@ public String toString()
     return defaultTimeout;
   }
 
+  public static <T> Query<T> setFailOnTruncatedResponseContext(Query<T> query)

Review comment:
       Should be `withFailOnTruncatedResponseContext`, not `setFailOnTruncatedResponseContext`, because nothing's being set (a modified copy is being returned).




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446451038



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -766,7 +766,6 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
                    .withDruidCluster(cluster)
                    .withLoadManagementPeons(loadManagementPeons)
                    .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withBalancerReferenceTimestamp(DateTimes.nowUtc())

Review comment:
       No, it's not related. I happened to find it not in use. I know it's better to not fix unrelated stuffs, but seems pretty trivial.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447167924



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Is deferring `baseRunner.run()` going to have any unintended consequences? For example, is it going to meaningfully delay initiation of connections to historicals? I'm not sure, but in general this area of the system is sensitive to when exactly things get called, so it would be good to look into this when changing it.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447520607



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       I suppose we can agree to disagree; I don't feel super strongly about this particular case since it's not a hot iterator, but I still think the test code in here complicates things overall and makes it harder to casually know what's going on, and wouldn't have done this in this way.

##########
File path: server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
##########
@@ -283,13 +301,23 @@ public CachingClusteredClient(
       return contextBuilder.build();
     }
 
-    Sequence<T> run(final UnaryOperator<TimelineLookup<String, ServerSelector>> timelineConverter)
+    /**
+     * Builds a query distribution and merge plan.
+     *
+     * This method returns an empty sequence if the query datasource is unknown or there is matching result-level cache.
+     * Otherwise, it creates a sequence merging sequences from the regular broker cache and remote servers. If parallel
+     * merge is enabled, it can merge and *combine* the underlying sequences in parallel.
+     *
+     * @return a pair of a sequence merging results from remote query servers and the number of remote servers
+     *         participating in query processing.
+     */
+    NonnullPair<Sequence<T>, Integer> run(final UnaryOperator<TimelineLookup<String, ServerSelector>> timelineConverter)

Review comment:
       nit: I think a dedicated type would be more appropriate than using a generic Pair, since it makes it a lot easier to follow what is going on than having to keep a mental model of what `lhs` and `rhs` are.
   
   Something like:
   ```
   class ClusterQueryResults<T>
   {
       Sequence<T> results;
       int numServers;
   }
   ```
   This would be much easier to understand, and has the added benefit of when we realize we need a 3rd thing we can just add it.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447174398



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostRelevantId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostRelevantId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);
+    }
+
+    final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
+    if (maybeMissingSegments == null) {
+      return Collections.emptyList();
+    }
+
+    return jsonMapper.convertValue(
+        maybeMissingSegments,
+        new TypeReference<List<SegmentDescriptor>>()
+        {
+        }
+    );
+  }
+
+  /**
+   * A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
+   * sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
+   * the sequences populated by this iterator will be merged (not combined) with the base sequence.
+   *
+   * The design of this iterator depends on how {@link MergeSequence} works; the MergeSequence pops an item from
+   * each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
+   * it pushes a new item from the sequence where the returned item was originally from. Since the first returned
+   * sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
+   * on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
+   * servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
+   * If there are missing segments reported, this iterator will rewrite the query with those reported segments and
+   * reissue the rewritten query.
+   *
+   * @see org.apache.druid.client.CachingClusteredClient
+   * @see org.apache.druid.client.DirectDruidClient
+   */
+  private class RetryingSequenceIterator implements Iterator<Sequence<T>>
+  {
+    private final QueryPlus<T> queryPlus;
+    private final ResponseContext context;
+    private final QueryRunner<T> baseQueryRunner;
+    private final Runnable runnableAfterFirstAttempt;
+
+    private boolean first = true;
+    private Sequence<T> sequence = null;
+    private int retryCount = 0;
+
+    private RetryingSequenceIterator(
+        QueryPlus<T> queryPlus,
+        ResponseContext context,
+        QueryRunner<T> baseQueryRunner,
+        Runnable runnableAfterFirstAttempt
+    )
+    {
+      this.queryPlus = queryPlus;
+      this.context = context;
+      this.baseQueryRunner = baseQueryRunner;
+      this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (first) {
+        sequence = baseQueryRunner.run(queryPlus, context);
+        // runnableAfterFirstAttempt is only for testing, it must be no-op for production code.
+        runnableAfterFirstAttempt.run();

Review comment:
       I don't necessarily agree that extending a production class in test code is cleaner than adding a test-only option to production code. They're equivalent in terms of test authenticity, and I'm biased towards thinking that extending things is bad in general (because it splits interconnected logic over two different files). So the test-only option sounds good to me.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r447167924



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());

Review comment:
       Is deferring `baseRunner.run()` going to have any unintended consequences? For example, is it going to delay initiation of connections to historicals? I'm not sure, but in general this area of the system is sensitive to when exactly things get called, so it would be good to look into this when changing it.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446451764



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest

Review comment:
       I agree the integration tests will be more useful but still want to keep the unit tests since it's way easier to debug. I am working on integration tests as commented in the PR description which will be added in a follow-up PR. BTW, I think the idea described in https://github.com/apache/druid/pull/10082#discussion_r446451237 seems useful for integration tests. That way, we can emulate the what can really happen in real clusters.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448076264



##########
File path: processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
##########
@@ -112,6 +114,30 @@
         "uncoveredIntervalsOverflowed",
             (oldValue, newValue) -> (boolean) oldValue || (boolean) newValue
     ),
+    /**
+     * Map of most relevant query ID to remaining number of responses from query nodes.
+     * The value is initialized in {@code CachingClusteredClient} when it initializes the connection to the query nodes,
+     * and is updated whenever they respond (@code DirectDruidClient). {@code RetryQueryRunner} uses this value to
+     * check if the {@link #MISSING_SEGMENTS} is valid.
+     *
+     * Currently, the broker doesn't run subqueries in parallel, the remaining number of responses will be updated
+     * one by one per subquery. However, since we are planning to parallelize running subqueries, we store them

Review comment:
       > we are planning to parallelize running subqueries
   
   I'm not sure if we are, but, we also haven't decided definitely _not_ to.
   
   I think it would also be okay to not handle parallel queries here, but instead build in a sanity check that verifies the subqueries are issued in series. Maybe by verifying that when the id changes, the previous number of responses remaining must be down to zero. It's up to you. I think your current code is okay too.

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    // Sanity check before retrieving missingSegments from responseContext.
+    // The missingSegments in the responseContext is only valid when all servers have responded to the broker.
+    // The remainingResponses must be not null but 0 in the responseContext at this point.
+    final ConcurrentHashMap<String, Integer> idToRemainingResponses =
+        (ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
+            context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
+            "%s in responseContext",
+            Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
+        );
+
+    final int remainingResponses = Preconditions.checkNotNull(
+        idToRemainingResponses.get(queryPlus.getQuery().getMostSpecificId()),
+        "Number of remaining responses for query[%s]",
+        queryPlus.getQuery().getMostSpecificId()
+    );
+    if (remainingResponses > 0) {
+      throw new ISE("Failed to check missing segments due to missing responds from [%d] servers", remainingResponses);

Review comment:
       If this happens, is it a bug? Or might this happen for some legitimate reason?
   
   If it is a bug: please include a comment that this message means there was a bug. (So people that get the message and search for it in the code will see that it is a sign of a bug.)
   
   If there could be a legitimate reason: in this case we should improve the error message to help the user understand what the legitimate reason might be.
   
   (a nit: spelling: should be "missing responses" rather than "missing responds".)

##########
File path: server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java
##########
@@ -329,15 +329,18 @@ private DataSource inlineIfNecessary(
         }
       } else if (canRunQueryUsingLocalWalker(subQuery) || canRunQueryUsingClusterWalker(subQuery)) {
         // Subquery needs to be inlined. Assign it a subquery id and run it.
-        final Query subQueryWithId = subQuery.withSubQueryId(UUID.randomUUID().toString());
+        final Query subQueryWithId = subQuery.withDefaultSubQueryId();
 
         final Sequence<?> queryResults;
 
         if (dryRun) {
           queryResults = Sequences.empty();
         } else {
           final QueryRunner subqueryRunner = subQueryWithId.getRunner(this);
-          queryResults = subqueryRunner.run(QueryPlus.wrap(subQueryWithId));
+          queryResults = subqueryRunner.run(
+              QueryPlus.wrap(subQueryWithId),
+              DirectDruidClient.makeResponseContextForQuery()

Review comment:
       Oops, not including a context here was a mistake, thanks for fixing it.
   
   Now that I think about it, though, making a new context here will mean we aren't going to properly return context from subqueries up to the original caller. This includes not reporting missing segments in the case where `RetryQueryRunnerConfig.isReturnPartialResults = true`.
   
   It would be better to share the context that was created in QueryLifecycle. Is it feasible to do this? Maybe by moving some of this logic to be lazy and happen inside the returned QueryRunner? (It will get a copy of the context.)
   
   Btw, this sounds like it might be tough to do, so we could also address it with documentation about known limitations. But I think we either need to fix it, or document it.

##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       I'm not sure where to put this comment, but I noticed that the `MISSING_SEGMENTS` array could get truncated by the historical that generated it; see `ResponseContext.serializeWith`. It looks like this was discussed in #2331. We can't allow this if we are going to rely on the missing segments list for query correctness. I think that means we need to introduce an option that tells the QueryResource that it should throw an error rather than truncate, and we should always set that option when communicating from the Broker to data servers.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r446451349



##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);
+    Assert.assertEquals(expectedTimeseriesResult(queryResult.size()), queryResult);
+  }
+
+  @Test
+  public void testRetryUntilWeGetFullResult()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(100, false), // retry up to 100
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertTrue(0 < queryRunner.getNumTotalRetries());

Review comment:
       See https://github.com/apache/druid/pull/10082#discussion_r446451237.

##########
File path: server/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.CachingClusteredClient;
+import org.apache.druid.client.DirectDruidClient;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.client.SimpleServerView;
+import org.apache.druid.client.TestHttpClient;
+import org.apache.druid.client.TestHttpClient.SimpleServerManager;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.ForegroundCachePopulator;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.guice.http.DruidHttpClientConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.SegmentMissingException;
+import org.apache.druid.segment.generator.GeneratorBasicSchemas;
+import org.apache.druid.segment.generator.GeneratorSchemaInfo;
+import org.apache.druid.segment.generator.SegmentGenerator;
+import org.apache.druid.server.QueryStackTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ForkJoinPool;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class RetryQueryRunnerTest
+{
+  private static final Closer CLOSER = Closer.create();
+  private static final String DATASOURCE = "datasource";
+  private static final GeneratorSchemaInfo SCHEMA_INFO = GeneratorBasicSchemas.SCHEMA_MAP.get("basic");
+  private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private final QueryToolChestWarehouse toolChestWarehouse;
+  private final QueryRunnerFactoryConglomerate conglomerate;
+
+  private SegmentGenerator segmentGenerator;
+  private TestHttpClient httpClient;
+  private SimpleServerView simpleServerView;
+  private CachingClusteredClient cachingClusteredClient;
+  private List<DruidServer> servers;
+
+  public RetryQueryRunnerTest()
+  {
+    conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER, USE_PARALLEL_MERGE_POOL_CONFIGURED);
+
+    toolChestWarehouse = new QueryToolChestWarehouse()
+    {
+      @Override
+      public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
+      {
+        return conglomerate.findFactory(query).getToolchest();
+      }
+    };
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws IOException
+  {
+    CLOSER.close();
+  }
+
+  @Before
+  public void setup()
+  {
+    segmentGenerator = new SegmentGenerator();
+    httpClient = new TestHttpClient(objectMapper);
+    simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
+    cachingClusteredClient = new CachingClusteredClient(
+        toolChestWarehouse,
+        simpleServerView,
+        MapCache.create(0),
+        objectMapper,
+        new ForegroundCachePopulator(objectMapper, new CachePopulatorStats(), 0),
+        new CacheConfig(),
+        new DruidHttpClientConfig(),
+        QueryStackTests.getProcessingConfig(USE_PARALLEL_MERGE_POOL_CONFIGURED),
+        ForkJoinPool.commonPool(),
+        QueryStackTests.DEFAULT_NOOP_SCHEDULER
+    );
+    servers = new ArrayList<>();
+  }
+
+  @After
+  public void tearDown() throws IOException
+  {
+    segmentGenerator.close();
+  }
+
+  private void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex)
+  {
+    servers.add(server);
+    simpleServerView.addServer(server, dataSegment);
+    httpClient.addServerAndRunner(server, new SimpleServerManager(conglomerate, dataSegment.getId(), queryableIndex));
+  }
+
+  @Test
+  public void testNoRetry()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, false),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(0, queryRunner.getNumTotalRetries());
+    Assert.assertFalse(queryResult.isEmpty());
+    Assert.assertEquals(expectedTimeseriesResult(10), queryResult);
+  }
+
+  @Test
+  public void testRetryForMovedSegment()
+  {
+    prepareCluster(10);
+    final TimeseriesQuery query = timeseriesQuery(SCHEMA_INFO.getDataInterval());
+    final RetryQueryRunner<Result<TimeseriesResultValue>> queryRunner = createQueryRunner(
+        newRetryQueryRunnerConfig(1, true),
+        query
+    );
+    final Sequence<Result<TimeseriesResultValue>> sequence = queryRunner.run(QueryPlus.wrap(query));
+
+    // Let's move a segment
+    dropSegmentFromServerAndAddNewServerForSegment(servers.get(0));
+
+    final List<Result<TimeseriesResultValue>> queryResult = sequence.toList();
+    Assert.assertEquals(1, queryRunner.getNumTotalRetries());
+    // Note that we dropped a segment from a server, but it's still announced in the server view.
+    // As a result, we may get the full result or not depending on what server will get the retry query.
+    // If we hit the same server, the query will return incomplete result.
+    Assert.assertTrue(queryResult.size() > 8);

Review comment:
       Changed to test exact sizes.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #10082: Fix RetryQueryRunner to actually do the job

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #10082:
URL: https://github.com/apache/druid/pull/10082#discussion_r448135794



##########
File path: server/src/main/java/org/apache/druid/query/RetryQueryRunner.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.query;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker;
+import org.apache.druid.java.util.common.guava.MergeSequence;
+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.YieldingAccumulator;
+import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.context.ResponseContext.Key;
+import org.apache.druid.segment.SegmentMissingException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+public class RetryQueryRunner<T> implements QueryRunner<T>
+{
+  private static final Logger LOG = new Logger(RetryQueryRunner.class);
+
+  private final QueryRunner<T> baseRunner;
+  private final BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn;
+  private final RetryQueryRunnerConfig config;
+  private final ObjectMapper jsonMapper;
+
+  /**
+   * Runnable executed after the broker creates query distribution tree for the first attempt. This is only
+   * for testing and must not be used in production code.
+   */
+  private final Runnable runnableAfterFirstAttempt;
+
+  private int totalNumRetries;
+
+  public RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper
+  )
+  {
+    this(baseRunner, retryRunnerCreateFn, config, jsonMapper, () -> {});
+  }
+
+  /**
+   * Constructor only for testing.
+   */
+  @VisibleForTesting
+  RetryQueryRunner(
+      QueryRunner<T> baseRunner,
+      BiFunction<Query<T>, List<SegmentDescriptor>, QueryRunner<T>> retryRunnerCreateFn,
+      RetryQueryRunnerConfig config,
+      ObjectMapper jsonMapper,
+      Runnable runnableAfterFirstAttempt
+  )
+  {
+    this.baseRunner = baseRunner;
+    this.retryRunnerCreateFn = retryRunnerCreateFn;
+    this.config = config;
+    this.jsonMapper = jsonMapper;
+    this.runnableAfterFirstAttempt = runnableAfterFirstAttempt;
+  }
+
+  @VisibleForTesting
+  int getTotalNumRetries()
+  {
+    return totalNumRetries;
+  }
+
+  @Override
+  public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
+  {
+    return new YieldingSequenceBase<T>()
+    {
+      @Override
+      public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
+      {
+        final Sequence<Sequence<T>> retryingSequence = new BaseSequence<>(
+            new IteratorMaker<Sequence<T>, RetryingSequenceIterator>()
+            {
+              @Override
+              public RetryingSequenceIterator make()
+              {
+                return new RetryingSequenceIterator(queryPlus, context, baseRunner, runnableAfterFirstAttempt);
+              }
+
+              @Override
+              public void cleanup(RetryingSequenceIterator iterFromMake)
+              {
+                totalNumRetries = iterFromMake.retryCount;
+              }
+            }
+        );
+        return new MergeSequence<>(queryPlus.getQuery().getResultOrdering(), retryingSequence)
+            .toYielder(initValue, accumulator);
+      }
+    };
+  }
+
+  private List<SegmentDescriptor> getMissingSegments(QueryPlus<T> queryPlus, final ResponseContext context)

Review comment:
       Hmm, I was aware of this issue, but ignored since the max size of the serialized responseContext is 7k. The missingSegments is a list of `SegmentDescriptor` which has an `interval`, a `version`, and a `partitionNumber`. Since its serialized size is about 80 bytes, each historical can report up to about 90 missing segments. I thought it wouldn't happen often, but well, I guess it depends on how your cluster is configured and how many segments the query reads.
   
   > I'm not sure what better we can really do without a fairly large refactoring involving making the broker keep track of which segments went to which direct druid client/sequence, and allowing historicals to basically give up if missing so many that the list would be truncated, returning an empty sequence and an indicator that **all** of the segments it was queried for should be retried.
   
   Hmm, I was thinking that we don't have to keep track of all segments, but can send a flag in the query context to historicals which makes the query failed immediately if the serialized responseContext is truncated. The broker will catch the failure and cancel the entire query. This won't retry on the truncated serialization, but I think it makes sense since it's a sort of failure of the retry system. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org