|  | @@ -9,19 +9,18 @@ import org.apache.lucene.search.TotalHits;
 | 
	
		
			
				|  |  |  import org.elasticsearch.ExceptionsHelper;
 | 
	
		
			
				|  |  |  import org.elasticsearch.ResourceNotFoundException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse;
 | 
	
		
			
				|  |  | -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
 | 
	
		
			
				|  |  | -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.get.GetResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.support.master.AcknowledgedResponse;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.node.DiscoveryNode;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.unit.TimeValue;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.xcontent.ContextParser;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.reindex.ReindexPlugin;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.shard.ShardId;
 | 
	
		
			
				|  |  |  import org.elasticsearch.plugins.Plugin;
 | 
	
		
			
				|  |  | -import org.elasticsearch.plugins.PluginsService;
 | 
	
		
			
				|  |  | +import org.elasticsearch.plugins.SearchPlugin;
 | 
	
		
			
				|  |  |  import org.elasticsearch.rest.RestStatus;
 | 
	
		
			
				|  |  | +import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
 | 
	
		
			
				|  |  |  import org.elasticsearch.search.builder.SearchSourceBuilder;
 | 
	
		
			
				|  |  |  import org.elasticsearch.tasks.TaskId;
 | 
	
		
			
				|  |  |  import org.elasticsearch.test.ESIntegTestCase;
 | 
	
	
		
			
				|  | @@ -34,18 +33,15 @@ import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
 | 
	
		
			
				|  |  |  import org.elasticsearch.xpack.ilm.IndexLifecycle;
 | 
	
		
			
				|  |  | +import org.junit.After;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import java.io.Closeable;
 | 
	
		
			
				|  |  |  import java.util.Arrays;
 | 
	
		
			
				|  |  |  import java.util.Collection;
 | 
	
		
			
				|  |  | -import java.util.Comparator;
 | 
	
		
			
				|  |  | +import java.util.Collections;
 | 
	
		
			
				|  |  |  import java.util.Iterator;
 | 
	
		
			
				|  |  | -import java.util.Map;
 | 
	
		
			
				|  |  | +import java.util.List;
 | 
	
		
			
				|  |  |  import java.util.concurrent.ExecutionException;
 | 
	
		
			
				|  |  | -import java.util.concurrent.atomic.AtomicInteger;
 | 
	
		
			
				|  |  | -import java.util.concurrent.atomic.AtomicReference;
 | 
	
		
			
				|  |  | -import java.util.function.Function;
 | 
	
		
			
				|  |  | -import java.util.stream.Collectors;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import static org.elasticsearch.xpack.search.AsyncSearch.INDEX;
 | 
	
		
			
				|  |  |  import static org.elasticsearch.xpack.search.AsyncSearchMaintenanceService.ASYNC_SEARCH_CLEANUP_INTERVAL_SETTING;
 | 
	
	
		
			
				|  | @@ -55,6 +51,31 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
 | 
	
		
			
				|  |  |  public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |      interface SearchResponseIterator extends Iterator<AsyncSearchResponse>, Closeable {}
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public static class SearchTestPlugin extends Plugin implements SearchPlugin {
 | 
	
		
			
				|  |  | +        public SearchTestPlugin() {}
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        @Override
 | 
	
		
			
				|  |  | +        public List<QuerySpec<?>> getQueries() {
 | 
	
		
			
				|  |  | +            return Collections.singletonList(new QuerySpec<>(BlockingQueryBuilder.NAME, in -> new BlockingQueryBuilder(in),
 | 
	
		
			
				|  |  | +                p -> {
 | 
	
		
			
				|  |  | +                    throw new IllegalStateException("not implemented");
 | 
	
		
			
				|  |  | +                }));
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        @Override
 | 
	
		
			
				|  |  | +        public List<AggregationSpec> getAggregations() {
 | 
	
		
			
				|  |  | +            return Collections.singletonList(new AggregationSpec(CancellingAggregationBuilder.NAME, CancellingAggregationBuilder::new,
 | 
	
		
			
				|  |  | +                (ContextParser<String, CancellingAggregationBuilder>) (p, c) -> {
 | 
	
		
			
				|  |  | +                    throw new IllegalStateException("not implemented");
 | 
	
		
			
				|  |  | +                }).addResultReader(InternalFilter::new));
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    @After
 | 
	
		
			
				|  |  | +    public void releaseQueryLatch() {
 | 
	
		
			
				|  |  | +        BlockingQueryBuilder.releaseQueryLatch();
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      @Override
 | 
	
		
			
				|  |  |      protected Collection<Class<? extends Plugin>> nodePlugins() {
 | 
	
		
			
				|  |  |          return Arrays.asList(LocalStateCompositeXPackPlugin.class, AsyncSearch.class, IndexLifecycle.class,
 | 
	
	
		
			
				|  | @@ -65,14 +86,14 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |      protected Settings nodeSettings(int nodeOrdinal) {
 | 
	
		
			
				|  |  |          return Settings.builder()
 | 
	
		
			
				|  |  |              .put(super.nodeSettings(0))
 | 
	
		
			
				|  |  | -            .put(ASYNC_SEARCH_CLEANUP_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(1))
 | 
	
		
			
				|  |  | +            .put(ASYNC_SEARCH_CLEANUP_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(100))
 | 
	
		
			
				|  |  |              .build();
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      /**
 | 
	
		
			
				|  |  |       * Restart the node that runs the {@link TaskId} decoded from the provided {@link AsyncExecutionId}.
 | 
	
		
			
				|  |  |       */
 | 
	
		
			
				|  |  | -    protected void restartTaskNode(String id) throws Exception {
 | 
	
		
			
				|  |  | +    protected void restartTaskNode(String id, String indexName) throws Exception {
 | 
	
		
			
				|  |  |          AsyncExecutionId searchId = AsyncExecutionId.decode(id);
 | 
	
		
			
				|  |  |          final ClusterStateResponse clusterState = client().admin().cluster()
 | 
	
		
			
				|  |  |              .prepareState().clear().setNodes(true).get();
 | 
	
	
		
			
				|  | @@ -83,7 +104,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |                  return super.onNodeStopped(nodeName);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  | -        ensureYellow(INDEX);
 | 
	
		
			
				|  |  | +        ensureYellow(INDEX, indexName);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      protected AsyncSearchResponse submitAsyncSearch(SubmitAsyncSearchRequest request) throws ExecutionException, InterruptedException {
 | 
	
	
		
			
				|  | @@ -147,41 +168,31 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    /**
 | 
	
		
			
				|  |  | +     * Returns a {@link SearchResponseIterator} that blocks query shard executions
 | 
	
		
			
				|  |  | +     * until {@link SearchResponseIterator#next()} is called. That allows to randomly
 | 
	
		
			
				|  |  | +     * generate partial results that can be consumed in order.
 | 
	
		
			
				|  |  | +     */
 | 
	
		
			
				|  |  |      protected SearchResponseIterator assertBlockingIterator(String indexName,
 | 
	
		
			
				|  |  | +                                                            int numShards,
 | 
	
		
			
				|  |  |                                                              SearchSourceBuilder source,
 | 
	
		
			
				|  |  |                                                              int numFailures,
 | 
	
		
			
				|  |  |                                                              int progressStep) throws Exception {
 | 
	
		
			
				|  |  |          SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(source, indexName);
 | 
	
		
			
				|  |  |          request.setBatchedReduceSize(progressStep);
 | 
	
		
			
				|  |  |          request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
 | 
	
		
			
				|  |  | -        ClusterSearchShardsResponse response = dataNodeClient().admin().cluster()
 | 
	
		
			
				|  |  | -            .prepareSearchShards(request.getSearchRequest().indices()).get();
 | 
	
		
			
				|  |  | -        AtomicInteger failures = new AtomicInteger(numFailures);
 | 
	
		
			
				|  |  | -        Map<ShardId, ShardIdLatch> shardLatchMap = Arrays.stream(response.getGroups())
 | 
	
		
			
				|  |  | -            .map(ClusterSearchShardsGroup::getShardId)
 | 
	
		
			
				|  |  | -            .collect(
 | 
	
		
			
				|  |  | -                Collectors.toMap(
 | 
	
		
			
				|  |  | -                    Function.identity(),
 | 
	
		
			
				|  |  | -                    id -> new ShardIdLatch(id, failures.decrementAndGet() >= 0)
 | 
	
		
			
				|  |  | -                )
 | 
	
		
			
				|  |  | -            );
 | 
	
		
			
				|  |  | -        ShardIdLatch[] shardLatchArray = shardLatchMap.values().stream()
 | 
	
		
			
				|  |  | -            .sorted(Comparator.comparing(ShardIdLatch::shardId))
 | 
	
		
			
				|  |  | -            .toArray(ShardIdLatch[]::new);
 | 
	
		
			
				|  |  | -        resetPluginsLatch(shardLatchMap);
 | 
	
		
			
				|  |  | -        request.getSearchRequest().source().query(new BlockingQueryBuilder(shardLatchMap));
 | 
	
		
			
				|  |  | +        BlockingQueryBuilder.QueryLatch queryLatch = BlockingQueryBuilder.acquireQueryLatch(numFailures);
 | 
	
		
			
				|  |  | +        request.getSearchRequest().source().query(new BlockingQueryBuilder(random().nextLong()));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          final AsyncSearchResponse initial = client().execute(SubmitAsyncSearchAction.INSTANCE, request).get();
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |          assertTrue(initial.isPartial());
 | 
	
		
			
				|  |  |          assertThat(initial.status(), equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  | -        assertThat(initial.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
 | 
	
		
			
				|  |  | +        assertThat(initial.getSearchResponse().getTotalShards(), equalTo(numShards));
 | 
	
		
			
				|  |  |          assertThat(initial.getSearchResponse().getSuccessfulShards(), equalTo(0));
 | 
	
		
			
				|  |  |          assertThat(initial.getSearchResponse().getShardFailures().length, equalTo(0));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          return new SearchResponseIterator() {
 | 
	
		
			
				|  |  |              private AsyncSearchResponse response = initial;
 | 
	
		
			
				|  |  | -            private int shardIndex = 0;
 | 
	
		
			
				|  |  |              private boolean isFirst = true;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
	
		
			
				|  | @@ -203,32 +214,24 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |                      isFirst = false;
 | 
	
		
			
				|  |  |                      return response;
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  | -                AtomicReference<AsyncSearchResponse> atomic = new AtomicReference<>();
 | 
	
		
			
				|  |  | -                int step = shardIndex == 0 ? progressStep+1 : progressStep-1;
 | 
	
		
			
				|  |  | -                int index = 0;
 | 
	
		
			
				|  |  | -                while (index < step && shardIndex < shardLatchArray.length) {
 | 
	
		
			
				|  |  | -                    if (shardLatchArray[shardIndex].shouldFail() == false) {
 | 
	
		
			
				|  |  | -                        ++index;
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                    shardLatchArray[shardIndex++].countDown();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | +                queryLatch.countDownAndReset();
 | 
	
		
			
				|  |  |                  AsyncSearchResponse newResponse = client().execute(GetAsyncSearchAction.INSTANCE,
 | 
	
		
			
				|  |  |                      new GetAsyncSearchAction.Request(response.getId())
 | 
	
		
			
				|  |  |                          .setWaitForCompletion(TimeValue.timeValueMillis(10))).get();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |                  if (newResponse.isRunning()) {
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.status(),  equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.status(), equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  |                      assertTrue(newResponse.isPartial());
 | 
	
		
			
				|  |  |                      assertNull(newResponse.getFailure());
 | 
	
		
			
				|  |  |                      assertNotNull(newResponse.getSearchResponse());
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(numShards));
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getShardFailures().length, lessThanOrEqualTo(numFailures));
 | 
	
		
			
				|  |  | -                } else if (numFailures == shardLatchArray.length) {
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.status(),  equalTo(RestStatus.INTERNAL_SERVER_ERROR));
 | 
	
		
			
				|  |  | +                } else if (numFailures == numShards) {
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.status(), equalTo(RestStatus.INTERNAL_SERVER_ERROR));
 | 
	
		
			
				|  |  |                      assertNotNull(newResponse.getFailure());
 | 
	
		
			
				|  |  |                      assertTrue(newResponse.isPartial());
 | 
	
		
			
				|  |  |                      assertNotNull(newResponse.getSearchResponse());
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(numShards));
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getSuccessfulShards(), equalTo(0));
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
 | 
	
		
			
				|  |  |                      assertNull(newResponse.getSearchResponse().getAggregations());
 | 
	
	
		
			
				|  | @@ -237,32 +240,22 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getHits().getTotalHits().relation,
 | 
	
		
			
				|  |  |                          equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
 | 
	
		
			
				|  |  |                  } else {
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.status(),  equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.status(), equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  |                      assertNotNull(newResponse.getSearchResponse());
 | 
	
		
			
				|  |  |                      assertFalse(newResponse.isPartial());
 | 
	
		
			
				|  |  |                      assertThat(newResponse.status(), equalTo(RestStatus.OK));
 | 
	
		
			
				|  |  | -                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
 | 
	
		
			
				|  |  | +                    assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(numShards));
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
 | 
	
		
			
				|  |  |                      assertThat(newResponse.getSearchResponse().getSuccessfulShards(),
 | 
	
		
			
				|  |  | -                        equalTo(shardLatchArray.length-newResponse.getSearchResponse().getShardFailures().length));
 | 
	
		
			
				|  |  | +                        equalTo(numShards - newResponse.getSearchResponse().getShardFailures().length));
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |                  return response = newResponse;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |              @Override
 | 
	
		
			
				|  |  |              public void close() {
 | 
	
		
			
				|  |  | -                Arrays.stream(shardLatchArray).forEach(shard -> {
 | 
	
		
			
				|  |  | -                    if (shard.getCount() == 1) {
 | 
	
		
			
				|  |  | -                        shard.countDown();
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                });
 | 
	
		
			
				|  |  | +                queryLatch.close();
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          };
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    private void resetPluginsLatch(Map<ShardId, ShardIdLatch> newLatch) {
 | 
	
		
			
				|  |  | -        for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) {
 | 
	
		
			
				|  |  | -            pluginsService.filterPlugins(SearchTestPlugin.class).forEach(p -> p.resetQueryLatch(newLatch));
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  |  }
 |