|
@@ -9,19 +9,18 @@ import org.apache.lucene.search.TotalHits;
|
|
import org.elasticsearch.ExceptionsHelper;
|
|
import org.elasticsearch.ExceptionsHelper;
|
|
import org.elasticsearch.ResourceNotFoundException;
|
|
import org.elasticsearch.ResourceNotFoundException;
|
|
import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse;
|
|
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.admin.cluster.state.ClusterStateResponse;
|
|
import org.elasticsearch.action.get.GetResponse;
|
|
import org.elasticsearch.action.get.GetResponse;
|
|
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
|
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
import org.elasticsearch.common.settings.Settings;
|
|
import org.elasticsearch.common.settings.Settings;
|
|
import org.elasticsearch.common.unit.TimeValue;
|
|
import org.elasticsearch.common.unit.TimeValue;
|
|
|
|
+import org.elasticsearch.common.xcontent.ContextParser;
|
|
import org.elasticsearch.index.reindex.ReindexPlugin;
|
|
import org.elasticsearch.index.reindex.ReindexPlugin;
|
|
-import org.elasticsearch.index.shard.ShardId;
|
|
|
|
import org.elasticsearch.plugins.Plugin;
|
|
import org.elasticsearch.plugins.Plugin;
|
|
-import org.elasticsearch.plugins.PluginsService;
|
|
|
|
|
|
+import org.elasticsearch.plugins.SearchPlugin;
|
|
import org.elasticsearch.rest.RestStatus;
|
|
import org.elasticsearch.rest.RestStatus;
|
|
|
|
+import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
|
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|
import org.elasticsearch.tasks.TaskId;
|
|
import org.elasticsearch.tasks.TaskId;
|
|
import org.elasticsearch.test.ESIntegTestCase;
|
|
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.SubmitAsyncSearchAction;
|
|
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
|
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
|
import org.elasticsearch.xpack.ilm.IndexLifecycle;
|
|
import org.elasticsearch.xpack.ilm.IndexLifecycle;
|
|
|
|
+import org.junit.After;
|
|
|
|
|
|
import java.io.Closeable;
|
|
import java.io.Closeable;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
-import java.util.Comparator;
|
|
|
|
|
|
+import java.util.Collections;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
-import java.util.Map;
|
|
|
|
|
|
+import java.util.List;
|
|
import java.util.concurrent.ExecutionException;
|
|
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.AsyncSearch.INDEX;
|
|
import static org.elasticsearch.xpack.search.AsyncSearchMaintenanceService.ASYNC_SEARCH_CLEANUP_INTERVAL_SETTING;
|
|
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 {
|
|
public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|
interface SearchResponseIterator extends Iterator<AsyncSearchResponse>, Closeable {}
|
|
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
|
|
@Override
|
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
|
return Arrays.asList(LocalStateCompositeXPackPlugin.class, AsyncSearch.class, IndexLifecycle.class,
|
|
return Arrays.asList(LocalStateCompositeXPackPlugin.class, AsyncSearch.class, IndexLifecycle.class,
|
|
@@ -65,14 +86,14 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|
protected Settings nodeSettings(int nodeOrdinal) {
|
|
protected Settings nodeSettings(int nodeOrdinal) {
|
|
return Settings.builder()
|
|
return Settings.builder()
|
|
.put(super.nodeSettings(0))
|
|
.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();
|
|
.build();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Restart the node that runs the {@link TaskId} decoded from the provided {@link AsyncExecutionId}.
|
|
* 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);
|
|
AsyncExecutionId searchId = AsyncExecutionId.decode(id);
|
|
final ClusterStateResponse clusterState = client().admin().cluster()
|
|
final ClusterStateResponse clusterState = client().admin().cluster()
|
|
.prepareState().clear().setNodes(true).get();
|
|
.prepareState().clear().setNodes(true).get();
|
|
@@ -83,7 +104,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|
return super.onNodeStopped(nodeName);
|
|
return super.onNodeStopped(nodeName);
|
|
}
|
|
}
|
|
});
|
|
});
|
|
- ensureYellow(INDEX);
|
|
|
|
|
|
+ ensureYellow(INDEX, indexName);
|
|
}
|
|
}
|
|
|
|
|
|
protected AsyncSearchResponse submitAsyncSearch(SubmitAsyncSearchRequest request) throws ExecutionException, InterruptedException {
|
|
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,
|
|
protected SearchResponseIterator assertBlockingIterator(String indexName,
|
|
|
|
+ int numShards,
|
|
SearchSourceBuilder source,
|
|
SearchSourceBuilder source,
|
|
int numFailures,
|
|
int numFailures,
|
|
int progressStep) throws Exception {
|
|
int progressStep) throws Exception {
|
|
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(source, indexName);
|
|
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(source, indexName);
|
|
request.setBatchedReduceSize(progressStep);
|
|
request.setBatchedReduceSize(progressStep);
|
|
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
|
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();
|
|
final AsyncSearchResponse initial = client().execute(SubmitAsyncSearchAction.INSTANCE, request).get();
|
|
-
|
|
|
|
assertTrue(initial.isPartial());
|
|
assertTrue(initial.isPartial());
|
|
assertThat(initial.status(), equalTo(RestStatus.OK));
|
|
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().getSuccessfulShards(), equalTo(0));
|
|
assertThat(initial.getSearchResponse().getShardFailures().length, equalTo(0));
|
|
assertThat(initial.getSearchResponse().getShardFailures().length, equalTo(0));
|
|
|
|
|
|
return new SearchResponseIterator() {
|
|
return new SearchResponseIterator() {
|
|
private AsyncSearchResponse response = initial;
|
|
private AsyncSearchResponse response = initial;
|
|
- private int shardIndex = 0;
|
|
|
|
private boolean isFirst = true;
|
|
private boolean isFirst = true;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -203,32 +214,24 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|
isFirst = false;
|
|
isFirst = false;
|
|
return response;
|
|
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,
|
|
AsyncSearchResponse newResponse = client().execute(GetAsyncSearchAction.INSTANCE,
|
|
new GetAsyncSearchAction.Request(response.getId())
|
|
new GetAsyncSearchAction.Request(response.getId())
|
|
.setWaitForCompletion(TimeValue.timeValueMillis(10))).get();
|
|
.setWaitForCompletion(TimeValue.timeValueMillis(10))).get();
|
|
|
|
|
|
if (newResponse.isRunning()) {
|
|
if (newResponse.isRunning()) {
|
|
- assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
|
|
|
|
|
+ assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
|
assertTrue(newResponse.isPartial());
|
|
assertTrue(newResponse.isPartial());
|
|
assertNull(newResponse.getFailure());
|
|
assertNull(newResponse.getFailure());
|
|
assertNotNull(newResponse.getSearchResponse());
|
|
assertNotNull(newResponse.getSearchResponse());
|
|
- assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
|
|
|
|
|
|
+ assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(numShards));
|
|
assertThat(newResponse.getSearchResponse().getShardFailures().length, lessThanOrEqualTo(numFailures));
|
|
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());
|
|
assertNotNull(newResponse.getFailure());
|
|
assertTrue(newResponse.isPartial());
|
|
assertTrue(newResponse.isPartial());
|
|
assertNotNull(newResponse.getSearchResponse());
|
|
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().getSuccessfulShards(), equalTo(0));
|
|
assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
|
|
assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
|
|
assertNull(newResponse.getSearchResponse().getAggregations());
|
|
assertNull(newResponse.getSearchResponse().getAggregations());
|
|
@@ -237,32 +240,22 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|
assertThat(newResponse.getSearchResponse().getHits().getTotalHits().relation,
|
|
assertThat(newResponse.getSearchResponse().getHits().getTotalHits().relation,
|
|
equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
|
|
equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
|
|
} else {
|
|
} else {
|
|
- assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
|
|
|
|
|
+ assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
|
assertNotNull(newResponse.getSearchResponse());
|
|
assertNotNull(newResponse.getSearchResponse());
|
|
assertFalse(newResponse.isPartial());
|
|
assertFalse(newResponse.isPartial());
|
|
assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
|
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().getShardFailures().length, equalTo(numFailures));
|
|
assertThat(newResponse.getSearchResponse().getSuccessfulShards(),
|
|
assertThat(newResponse.getSearchResponse().getSuccessfulShards(),
|
|
- equalTo(shardLatchArray.length-newResponse.getSearchResponse().getShardFailures().length));
|
|
|
|
|
|
+ equalTo(numShards - newResponse.getSearchResponse().getShardFailures().length));
|
|
}
|
|
}
|
|
return response = newResponse;
|
|
return response = newResponse;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void close() {
|
|
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));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
}
|
|
}
|