msearch: Cap the number of searches the msearch api will concurrently execute
By default the number of searches msearch executes is capped by the number of nodes multiplied with the default size of the search threadpool. This default can be overwritten by using the newly added `max_concurrent_searches` parameter. Before the msearch api would concurrently execute all searches concurrently. If many large msearch requests would be executed this could lead to some searches being rejected while other searches in the msearch request would succeed. The goal of this change is to avoid this exhausting of the search TP. Closes #17926
This commit is contained in:
parent
7c8eb184ee
commit
3b96055b23
|
@ -38,6 +38,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||||
*/
|
*/
|
||||||
public class MultiSearchRequest extends ActionRequest<MultiSearchRequest> implements CompositeIndicesRequest {
|
public class MultiSearchRequest extends ActionRequest<MultiSearchRequest> implements CompositeIndicesRequest {
|
||||||
|
|
||||||
|
private int maxConcurrentSearchRequests = 0;
|
||||||
private List<SearchRequest> requests = new ArrayList<>();
|
private List<SearchRequest> requests = new ArrayList<>();
|
||||||
|
|
||||||
private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed();
|
private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed();
|
||||||
|
@ -60,6 +61,25 @@ public class MultiSearchRequest extends ActionRequest<MultiSearchRequest> implem
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the amount of search requests specified in this multi search requests are allowed to be ran concurrently.
|
||||||
|
*/
|
||||||
|
public int maxConcurrentSearchRequests() {
|
||||||
|
return maxConcurrentSearchRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets how many search requests specified in this multi search requests are allowed to be ran concurrently.
|
||||||
|
*/
|
||||||
|
public MultiSearchRequest maxConcurrentSearchRequests(int maxConcurrentSearchRequests) {
|
||||||
|
if (maxConcurrentSearchRequests < 1) {
|
||||||
|
throw new IllegalArgumentException("maxConcurrentSearchRequests must be positive");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.maxConcurrentSearchRequests = maxConcurrentSearchRequests;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public List<SearchRequest> requests() {
|
public List<SearchRequest> requests() {
|
||||||
return this.requests;
|
return this.requests;
|
||||||
}
|
}
|
||||||
|
@ -100,6 +120,7 @@ public class MultiSearchRequest extends ActionRequest<MultiSearchRequest> implem
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
|
maxConcurrentSearchRequests = in.readVInt();
|
||||||
int size = in.readVInt();
|
int size = in.readVInt();
|
||||||
for (int i = 0; i < size; i++) {
|
for (int i = 0; i < size; i++) {
|
||||||
SearchRequest request = new SearchRequest();
|
SearchRequest request = new SearchRequest();
|
||||||
|
@ -111,6 +132,7 @@ public class MultiSearchRequest extends ActionRequest<MultiSearchRequest> implem
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
|
out.writeVInt(maxConcurrentSearchRequests);
|
||||||
out.writeVInt(requests.size());
|
out.writeVInt(requests.size());
|
||||||
for (SearchRequest request : requests) {
|
for (SearchRequest request : requests) {
|
||||||
request.writeTo(out);
|
request.writeTo(out);
|
||||||
|
|
|
@ -71,4 +71,12 @@ public class MultiSearchRequestBuilder extends ActionRequestBuilder<MultiSearchR
|
||||||
request().indicesOptions(indicesOptions);
|
request().indicesOptions(indicesOptions);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets how many search requests specified in this multi search requests are allowed to be ran concurrently.
|
||||||
|
*/
|
||||||
|
public MultiSearchRequestBuilder setMaxConcurrentSearchRequests(int maxConcurrentSearchRequests) {
|
||||||
|
request().maxConcurrentSearchRequests(maxConcurrentSearchRequests);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.action.search;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.HandledTransportAction;
|
import org.elasticsearch.action.support.HandledTransportAction;
|
||||||
|
import org.elasticsearch.action.support.TransportAction;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
@ -29,17 +30,20 @@ import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||||
|
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||||
|
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class TransportMultiSearchAction extends HandledTransportAction<MultiSearchRequest, MultiSearchResponse> {
|
public class TransportMultiSearchAction extends HandledTransportAction<MultiSearchRequest, MultiSearchResponse> {
|
||||||
|
|
||||||
|
private final int availableProcessors;
|
||||||
private final ClusterService clusterService;
|
private final ClusterService clusterService;
|
||||||
private final TransportSearchAction searchAction;
|
private final TransportAction<SearchRequest, SearchResponse> searchAction;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TransportMultiSearchAction(Settings settings, ThreadPool threadPool, TransportService transportService,
|
public TransportMultiSearchAction(Settings settings, ThreadPool threadPool, TransportService transportService,
|
||||||
|
@ -48,38 +52,96 @@ public class TransportMultiSearchAction extends HandledTransportAction<MultiSear
|
||||||
super(settings, MultiSearchAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, MultiSearchRequest::new);
|
super(settings, MultiSearchAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, MultiSearchRequest::new);
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
this.searchAction = searchAction;
|
this.searchAction = searchAction;
|
||||||
|
this.availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
// For testing only:
|
||||||
|
TransportMultiSearchAction(ThreadPool threadPool, ActionFilters actionFilters, TransportService transportService,
|
||||||
|
ClusterService clusterService, TransportAction<SearchRequest, SearchResponse> searchAction,
|
||||||
|
IndexNameExpressionResolver indexNameExpressionResolver, int availableProcessors) {
|
||||||
|
super(Settings.EMPTY, MultiSearchAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, MultiSearchRequest::new);
|
||||||
|
this.clusterService = clusterService;
|
||||||
|
this.searchAction = searchAction;
|
||||||
|
this.availableProcessors = availableProcessors;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(final MultiSearchRequest request, final ActionListener<MultiSearchResponse> listener) {
|
protected void doExecute(MultiSearchRequest request, ActionListener<MultiSearchResponse> listener) {
|
||||||
ClusterState clusterState = clusterService.state();
|
ClusterState clusterState = clusterService.state();
|
||||||
clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ);
|
clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ);
|
||||||
|
|
||||||
final AtomicArray<MultiSearchResponse.Item> responses = new AtomicArray<>(request.requests().size());
|
int maxConcurrentSearches = request.maxConcurrentSearchRequests();
|
||||||
final AtomicInteger counter = new AtomicInteger(responses.length());
|
if (maxConcurrentSearches == 0) {
|
||||||
for (int i = 0; i < responses.length(); i++) {
|
maxConcurrentSearches = defaultMaxConcurrentSearches(availableProcessors, clusterState);
|
||||||
final int index = i;
|
}
|
||||||
searchAction.execute(request.requests().get(i), new ActionListener<SearchResponse>() {
|
|
||||||
|
Queue<SearchRequestSlot> searchRequestSlots = new ConcurrentLinkedQueue<>();
|
||||||
|
for (int i = 0; i < request.requests().size(); i++) {
|
||||||
|
SearchRequest searchRequest = request.requests().get(i);
|
||||||
|
searchRequestSlots.add(new SearchRequestSlot(searchRequest, i));
|
||||||
|
}
|
||||||
|
|
||||||
|
int numRequests = request.requests().size();
|
||||||
|
final AtomicArray<MultiSearchResponse.Item> responses = new AtomicArray<>(numRequests);
|
||||||
|
final AtomicInteger responseCounter = new AtomicInteger(numRequests);
|
||||||
|
int numConcurrentSearches = Math.min(numRequests, maxConcurrentSearches);
|
||||||
|
for (int i = 0; i < numConcurrentSearches; i++) {
|
||||||
|
executeSearch(searchRequestSlots, responses, responseCounter, listener);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This is not perfect and makes a big assumption, that all nodes have the same thread pool size / have the number
|
||||||
|
* of processors and that shard of the indices the search requests go to are more or less evenly distributed across
|
||||||
|
* all nodes in the cluster. But I think it is a good enough default for most cases, if not then the default should be
|
||||||
|
* overwritten in the request itself.
|
||||||
|
*/
|
||||||
|
static int defaultMaxConcurrentSearches(int availableProcessors, ClusterState state) {
|
||||||
|
int numDateNodes = state.getNodes().getDataNodes().size();
|
||||||
|
// availableProcessors will never be larger than 32, so max defaultMaxConcurrentSearches will never be larger than 49,
|
||||||
|
// but we don't know about about other search requests that are being executed so lets cap at 10 per node
|
||||||
|
int defaultSearchThreadPoolSize = Math.min(ThreadPool.searchThreadPoolSize(availableProcessors), 10);
|
||||||
|
return Math.max(1, numDateNodes * defaultSearchThreadPoolSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeSearch(Queue<SearchRequestSlot> requests, AtomicArray<MultiSearchResponse.Item> responses,
|
||||||
|
AtomicInteger responseCounter, ActionListener<MultiSearchResponse> listener) {
|
||||||
|
SearchRequestSlot request = requests.poll();
|
||||||
|
if (request == null) {
|
||||||
|
// Ok... so there're no more requests then this is ok, we're then waiting for running requests to complete
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
searchAction.execute(request.request, new ActionListener<SearchResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(SearchResponse searchResponse) {
|
public void onResponse(SearchResponse searchResponse) {
|
||||||
responses.set(index, new MultiSearchResponse.Item(searchResponse, null));
|
responses.set(request.responseSlot, new MultiSearchResponse.Item(searchResponse, null));
|
||||||
if (counter.decrementAndGet() == 0) {
|
handleResponse();
|
||||||
finishHim();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onFailure(Throwable e) {
|
public void onFailure(Throwable e) {
|
||||||
responses.set(index, new MultiSearchResponse.Item(null, e));
|
responses.set(request.responseSlot, new MultiSearchResponse.Item(null, e));
|
||||||
if (counter.decrementAndGet() == 0) {
|
handleResponse();
|
||||||
finishHim();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void finishHim() {
|
private void handleResponse() {
|
||||||
|
if (responseCounter.decrementAndGet() == 0) {
|
||||||
listener.onResponse(new MultiSearchResponse(responses.toArray(new MultiSearchResponse.Item[responses.length()])));
|
listener.onResponse(new MultiSearchResponse(responses.toArray(new MultiSearchResponse.Item[responses.length()])));
|
||||||
|
} else {
|
||||||
|
executeSearch(requests, responses, responseCounter, listener);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final static class SearchRequestSlot {
|
||||||
|
|
||||||
|
final SearchRequest request;
|
||||||
|
final int responseSlot;
|
||||||
|
|
||||||
|
SearchRequestSlot(SearchRequest request, int responseSlot) {
|
||||||
|
this.request = request;
|
||||||
|
this.responseSlot = responseSlot;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -91,6 +91,9 @@ public class RestMultiSearchAction extends BaseRestHandler {
|
||||||
@Override
|
@Override
|
||||||
public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) throws Exception {
|
public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) throws Exception {
|
||||||
MultiSearchRequest multiSearchRequest = new MultiSearchRequest();
|
MultiSearchRequest multiSearchRequest = new MultiSearchRequest();
|
||||||
|
if (request.hasParam("max_concurrent_searches")) {
|
||||||
|
multiSearchRequest.maxConcurrentSearchRequests(request.paramAsInt("max_concurrent_searches", 0));
|
||||||
|
}
|
||||||
|
|
||||||
String[] indices = Strings.splitStringByCommaToArray(request.param("index"));
|
String[] indices = Strings.splitStringByCommaToArray(request.param("index"));
|
||||||
String[] types = Strings.splitStringByCommaToArray(request.param("type"));
|
String[] types = Strings.splitStringByCommaToArray(request.param("type"));
|
||||||
|
|
|
@ -167,7 +167,7 @@ public class ThreadPool extends AbstractComponent implements Closeable {
|
||||||
builders.put(Names.INDEX, new FixedExecutorBuilder(settings, Names.INDEX, availableProcessors, 200));
|
builders.put(Names.INDEX, new FixedExecutorBuilder(settings, Names.INDEX, availableProcessors, 200));
|
||||||
builders.put(Names.BULK, new FixedExecutorBuilder(settings, Names.BULK, availableProcessors, 50));
|
builders.put(Names.BULK, new FixedExecutorBuilder(settings, Names.BULK, availableProcessors, 50));
|
||||||
builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, availableProcessors, 1000));
|
builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, availableProcessors, 1000));
|
||||||
builders.put(Names.SEARCH, new FixedExecutorBuilder(settings, Names.SEARCH, ((availableProcessors * 3) / 2) + 1, 1000));
|
builders.put(Names.SEARCH, new FixedExecutorBuilder(settings, Names.SEARCH, searchThreadPoolSize(availableProcessors), 1000));
|
||||||
builders.put(Names.MANAGEMENT, new ScalingExecutorBuilder(Names.MANAGEMENT, 1, 5, TimeValue.timeValueMinutes(5)));
|
builders.put(Names.MANAGEMENT, new ScalingExecutorBuilder(Names.MANAGEMENT, 1, 5, TimeValue.timeValueMinutes(5)));
|
||||||
// no queue as this means clients will need to handle rejections on listener queue even if the operation succeeded
|
// no queue as this means clients will need to handle rejections on listener queue even if the operation succeeded
|
||||||
// the assumption here is that the listeners should be very lightweight on the listeners side
|
// the assumption here is that the listeners should be very lightweight on the listeners side
|
||||||
|
@ -389,6 +389,10 @@ public class ThreadPool extends AbstractComponent implements Closeable {
|
||||||
return boundedBy(2 * numberOfProcessors, 2, Integer.MAX_VALUE);
|
return boundedBy(2 * numberOfProcessors, 2, Integer.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static int searchThreadPoolSize(int availableProcessors) {
|
||||||
|
return ((availableProcessors * 3) / 2) + 1;
|
||||||
|
}
|
||||||
|
|
||||||
class LoggingRunnable implements Runnable {
|
class LoggingRunnable implements Runnable {
|
||||||
|
|
||||||
private final Runnable runnable;
|
private final Runnable runnable;
|
||||||
|
|
|
@ -22,8 +22,6 @@ package org.elasticsearch.action.search;
|
||||||
import org.elasticsearch.action.support.IndicesOptions;
|
import org.elasticsearch.action.support.IndicesOptions;
|
||||||
import org.elasticsearch.common.ParseFieldMatcher;
|
import org.elasticsearch.common.ParseFieldMatcher;
|
||||||
import org.elasticsearch.common.bytes.BytesArray;
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
import org.elasticsearch.common.collect.Tuple;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.common.xcontent.ToXContent;
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
@ -37,7 +35,6 @@ import org.elasticsearch.test.StreamsUtils;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import static java.util.Collections.singletonMap;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.nullValue;
|
import static org.hamcrest.Matchers.nullValue;
|
||||||
|
|
||||||
|
@ -167,6 +164,13 @@ public class MultiSearchRequestTests extends ESTestCase {
|
||||||
builder.string());
|
builder.string());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMaxConcurrentSearchRequests() {
|
||||||
|
MultiSearchRequest request = new MultiSearchRequest();
|
||||||
|
request.maxConcurrentSearchRequests(randomIntBetween(1, Integer.MAX_VALUE));
|
||||||
|
expectThrows(IllegalArgumentException.class, () ->
|
||||||
|
request.maxConcurrentSearchRequests(randomIntBetween(Integer.MIN_VALUE, 0)));
|
||||||
|
}
|
||||||
|
|
||||||
private IndicesQueriesRegistry registry() {
|
private IndicesQueriesRegistry registry() {
|
||||||
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
|
IndicesQueriesRegistry registry = new IndicesQueriesRegistry();
|
||||||
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
|
QueryParser<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
|
||||||
|
|
|
@ -0,0 +1,137 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch 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.elasticsearch.action.search;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.ActionFilter;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.TransportAction;
|
||||||
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.transport.LocalTransportAddress;
|
||||||
|
import org.elasticsearch.tasks.TaskManager;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.hamcrest.Matchers.nullValue;
|
||||||
|
import static org.hamcrest.Matchers.sameInstance;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
public class TransportMultiSearchActionTests extends ESTestCase {
|
||||||
|
|
||||||
|
public void testBatchExecute() throws Exception {
|
||||||
|
// Initialize depedencies of TransportMultiSearchAction
|
||||||
|
Settings settings = Settings.builder()
|
||||||
|
.put("node.name", TransportMultiSearchActionTests.class.getSimpleName())
|
||||||
|
.build();
|
||||||
|
ActionFilters actionFilters = mock(ActionFilters.class);
|
||||||
|
when(actionFilters.filters()).thenReturn(new ActionFilter[0]);
|
||||||
|
ThreadPool threadPool = new ThreadPool(settings);
|
||||||
|
TaskManager taskManager = mock(TaskManager.class);
|
||||||
|
TransportService transportService = mock(TransportService.class);
|
||||||
|
when(transportService.getTaskManager()).thenReturn(taskManager);
|
||||||
|
ClusterService clusterService = mock(ClusterService.class);
|
||||||
|
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("test")).build());
|
||||||
|
IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(Settings.EMPTY);
|
||||||
|
|
||||||
|
// Keep track of the number of concurrent searches started by multi search api,
|
||||||
|
// and if there are more searches than is allowed create an error and remember that.
|
||||||
|
int maxAllowedConcurrentSearches = scaledRandomIntBetween(1, 20);
|
||||||
|
AtomicInteger counter = new AtomicInteger();
|
||||||
|
AtomicReference<AssertionError> errorHolder = new AtomicReference<>();
|
||||||
|
TransportAction<SearchRequest, SearchResponse> searchAction = new TransportAction<SearchRequest, SearchResponse>
|
||||||
|
(Settings.EMPTY, "action", threadPool, actionFilters, resolver, taskManager) {
|
||||||
|
@Override
|
||||||
|
protected void doExecute(SearchRequest request, ActionListener<SearchResponse> listener) {
|
||||||
|
int currentConcurrentSearches = counter.incrementAndGet();
|
||||||
|
if (currentConcurrentSearches > maxAllowedConcurrentSearches) {
|
||||||
|
errorHolder.set(new AssertionError("Current concurrent search [" + currentConcurrentSearches +
|
||||||
|
"] is higher than is allowed [" + maxAllowedConcurrentSearches + "]"));
|
||||||
|
}
|
||||||
|
threadPool.executor(ThreadPool.Names.GENERIC).execute(
|
||||||
|
() -> {
|
||||||
|
try {
|
||||||
|
Thread.sleep(scaledRandomIntBetween(10, 1000));
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
}
|
||||||
|
counter.decrementAndGet();
|
||||||
|
listener.onResponse(new SearchResponse());
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
TransportMultiSearchAction action =
|
||||||
|
new TransportMultiSearchAction(threadPool, actionFilters, transportService, clusterService, searchAction, resolver, 10);
|
||||||
|
|
||||||
|
// Execute the multi search api and fail if we find an error after executing:
|
||||||
|
try {
|
||||||
|
int numSearchRequests = randomIntBetween(16, 128);
|
||||||
|
MultiSearchRequest multiSearchRequest = new MultiSearchRequest();
|
||||||
|
multiSearchRequest.maxConcurrentSearchRequests(maxAllowedConcurrentSearches);
|
||||||
|
for (int i = 0; i < numSearchRequests; i++) {
|
||||||
|
multiSearchRequest.add(new SearchRequest());
|
||||||
|
}
|
||||||
|
|
||||||
|
MultiSearchResponse response = action.execute(multiSearchRequest).actionGet();
|
||||||
|
assertThat(response.getResponses().length, equalTo(numSearchRequests));
|
||||||
|
assertThat(errorHolder.get(), nullValue());
|
||||||
|
} finally {
|
||||||
|
assertTrue(ESTestCase.terminate(threadPool));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDefaultMaxConcurrentSearches() {
|
||||||
|
int numDataNodes = randomIntBetween(1, 10);
|
||||||
|
DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
|
||||||
|
for (int i = 0; i < numDataNodes; i++) {
|
||||||
|
builder.put(new DiscoveryNode("_id" + i, new LocalTransportAddress("_id" + i), Collections.emptyMap(),
|
||||||
|
Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT));
|
||||||
|
}
|
||||||
|
builder.put(new DiscoveryNode("master", new LocalTransportAddress("mater"), Collections.emptyMap(),
|
||||||
|
Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT));
|
||||||
|
builder.put(new DiscoveryNode("ingest", new LocalTransportAddress("ingest"), Collections.emptyMap(),
|
||||||
|
Collections.singleton(DiscoveryNode.Role.INGEST), Version.CURRENT));
|
||||||
|
|
||||||
|
ClusterState state = ClusterState.builder(new ClusterName("_name")).nodes(builder).build();
|
||||||
|
int result = TransportMultiSearchAction.defaultMaxConcurrentSearches(10, state);
|
||||||
|
assertThat(result, equalTo(10 * numDataNodes));
|
||||||
|
|
||||||
|
state = ClusterState.builder(new ClusterName("_name")).build();
|
||||||
|
result = TransportMultiSearchAction.defaultMaxConcurrentSearches(10, state);
|
||||||
|
assertThat(result, equalTo(1));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.search.msearch;
|
package org.elasticsearch.search.msearch;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||||
import org.elasticsearch.index.query.QueryBuilders;
|
import org.elasticsearch.index.query.QueryBuilders;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
@ -29,9 +30,8 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFa
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
/**
|
public class MultiSearchIT extends ESIntegTestCase {
|
||||||
*/
|
|
||||||
public class SimpleMultiSearchIT extends ESIntegTestCase {
|
|
||||||
public void testSimpleMultiSearch() {
|
public void testSimpleMultiSearch() {
|
||||||
createIndex("test");
|
createIndex("test");
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
|
@ -54,4 +54,30 @@ public class SimpleMultiSearchIT extends ESIntegTestCase {
|
||||||
assertFirstHit(response.getResponses()[0].getResponse(), hasId("1"));
|
assertFirstHit(response.getResponses()[0].getResponse(), hasId("1"));
|
||||||
assertFirstHit(response.getResponses()[1].getResponse(), hasId("2"));
|
assertFirstHit(response.getResponses()[1].getResponse(), hasId("2"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testSimpleMultiSearchMoreRequests() {
|
||||||
|
createIndex("test");
|
||||||
|
int numDocs = randomIntBetween(0, 16);
|
||||||
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
client().prepareIndex("test", "type", Integer.toString(i)).setSource("{}").get();
|
||||||
|
}
|
||||||
|
refresh();
|
||||||
|
|
||||||
|
int numSearchRequests = randomIntBetween(0, 64);
|
||||||
|
MultiSearchRequest request = new MultiSearchRequest();
|
||||||
|
if (randomBoolean()) {
|
||||||
|
request.maxConcurrentSearchRequests(randomIntBetween(1, numSearchRequests));
|
||||||
|
}
|
||||||
|
for (int i = 0; i < numSearchRequests; i++) {
|
||||||
|
request.add(client().prepareSearch("test"));
|
||||||
|
}
|
||||||
|
|
||||||
|
MultiSearchResponse response = client().multiSearch(request).actionGet();
|
||||||
|
assertThat(response.getResponses().length, equalTo(numSearchRequests));
|
||||||
|
for (MultiSearchResponse.Item item : response) {
|
||||||
|
assertNoFailures(item.getResponse());
|
||||||
|
assertHitCount(item.getResponse(), numDocs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -71,6 +71,10 @@ against the `test2` index.
|
||||||
The `search_type` can be set in a similar manner to globally apply to
|
The `search_type` can be set in a similar manner to globally apply to
|
||||||
all search requests.
|
all search requests.
|
||||||
|
|
||||||
|
The msearch's `max_concurrent_searches` request parameter can be used to control
|
||||||
|
the maximum number of concurrent searches the multi search api will execute.
|
||||||
|
This default is based on the number of data nodes and the default search thread pool size.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[msearch-security]]
|
[[msearch-security]]
|
||||||
=== Security
|
=== Security
|
||||||
|
|
|
@ -20,6 +20,10 @@
|
||||||
"type" : "enum",
|
"type" : "enum",
|
||||||
"options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch"],
|
"options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch"],
|
||||||
"description" : "Search operation type"
|
"description" : "Search operation type"
|
||||||
|
},
|
||||||
|
"max_concurrent_searches" : {
|
||||||
|
"type" : "number",
|
||||||
|
"description" : "Controls the maximum number of concurrent searches the multi search api will execute"
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
Loading…
Reference in New Issue