From 76b467aa770980d015dd8769283e2b6c889850b5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 30 Sep 2021 09:06:59 +0200 Subject: [PATCH 01/28] WIP --- .../action/search/CanMatchNodeResponse.java | 41 ++ .../action/search/CanMatchPhase.java | 353 ++++++++++++++++++ .../action/search/CanMatchRequest.java | 231 ++++++++++++ .../action/search/SearchPhase.java | 4 + .../action/search/SearchTransportService.java | 12 + .../action/search/TransportSearchAction.java | 9 +- .../elasticsearch/search/SearchService.java | 15 + .../search/internal/ShardSearchRequest.java | 2 +- 8 files changed, 662 insertions(+), 5 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java create mode 100644 server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java create mode 100644 server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java new file mode 100644 index 0000000000000..c79c7b154f9c0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java @@ -0,0 +1,41 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.search; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.transport.TransportResponse; + +import java.io.IOException; +import java.util.List; + +public class CanMatchNodeResponse extends TransportResponse { + + private final List responses; + + public CanMatchNodeResponse(StreamInput in) throws IOException { + super(in); + responses = in.readList(SearchService.CanMatchResponse::new); + } + + public CanMatchNodeResponse(List responses) { + this.responses = responses; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeList(responses); + } + + public List getResponses() { + return responses; + } + +} diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java new file mode 100644 index 0000000000000..e0a37ba721298 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -0,0 +1,353 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.search; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.util.CollectionUtil; +import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.InternalSearchResponse; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.MinAndMax; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.transport.Transport; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.elasticsearch.core.Types.forciblyCast; + +public class CanMatchPhase extends SearchPhase { + + private final Logger logger; + private final SearchRequest request; + private final GroupShardsIterator shardsIts; + private final ActionListener listener; + private final SearchResponse.Clusters clusters; + private final TransportSearchAction.SearchTimeProvider timeProvider; + private final BiFunction nodeIdToConnection; + private final SearchTransportService searchTransportService; + private final Map shardItIndexMap; + private final Map concreteIndexBoosts; + private final Map aliasFilter; + + public CanMatchPhase(Logger logger, SearchTransportService searchTransportService, + BiFunction nodeIdToConnection, + Map aliasFilter, Map concreteIndexBoosts, + Executor executor, SearchRequest request, + ActionListener listener, GroupShardsIterator shardsIts, + TransportSearchAction.SearchTimeProvider timeProvider, ClusterState clusterState, + SearchTask task, Function, SearchPhase> phaseFactory, + SearchResponse.Clusters clusters, CoordinatorRewriteContextProvider coordinatorRewriteContextProvider) { + super("can_match"); + this.logger = logger; + this.searchTransportService = searchTransportService; + this.nodeIdToConnection = nodeIdToConnection; + this.request = request; + this.listener = listener; + this.shardsIts = shardsIts; + this.clusters = clusters; + this.timeProvider = timeProvider; + this.concreteIndexBoosts = concreteIndexBoosts; + this.aliasFilter = aliasFilter; + this.shardItIndexMap = new HashMap<>(); + + // we compute the shard index based on the natural order of the shards + // that participate in the search request. This means that this number is + // consistent between two requests that target the same shards. + List naturalOrder = new ArrayList<>(); + shardsIts.iterator().forEachRemaining(naturalOrder::add); + CollectionUtil.timSort(naturalOrder); + for (int i = 0; i < naturalOrder.size(); i++) { + shardItIndexMap.put(naturalOrder.get(i), i); + } + } + + @Override + public void run() throws IOException { + if (shardsIts.size() > 0) { + assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (request.allowPartialSearchResults() == false) { + final StringBuilder missingShards = new StringBuilder(); + // Fail-fast verification of all shards being available + for (int index = 0; index < shardsIts.size(); index++) { + final SearchShardIterator shardRoutings = shardsIts.get(index); + if (shardRoutings.size() == 0) { + if(missingShards.length() > 0){ + missingShards.append(", "); + } + missingShards.append(shardRoutings.shardId()); + } + } + if (missingShards.length() > 0) { + //Status red - shard is missing all copies and would produce partial results for an index search + final String msg = "Search rejected due to missing shards ["+ missingShards + + "]. Consider using `allow_partial_search_results` setting to bypass this error."; + throw new SearchPhaseExecutionException(getName(), msg, null, ShardSearchFailure.EMPTY_ARRAY); + } + } + Version version = request.minCompatibleShardNode(); + if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { + if (checkMinimumVersion(shardsIts) == false) { + throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", + request.minCompatibleShardNode()); + } + } + runRound(); + } + } + + private void runRound() { + // create CanMatchRequests for the given round + Map> requests = new HashMap<>(); + for (int i = 0; i < shardsIts.size(); i++) { + final SearchShardIterator shardRoutings = shardsIts.get(i); + assert shardRoutings.skip() == false; +// assert shardItIndexMap.containsKey(shardRoutings); +// int shardIndex = shardItIndexMap.get(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(target.getNodeId(), t -> new ArrayList<>()).add(target); + } + + + //performPhaseOnShard(shardIndex, shardRoutings, shardRoutings.nextOrNull()); + } + + for (Map.Entry> entry : requests.entrySet()) { + //new CanMatchRequest(getOriginalIndices(), request); + } + } + + private static final float DEFAULT_INDEX_BOOST = 1.0f; + + public final CanMatchRequest.ShardLevelRequest buildShardLevelRequest(SearchShardIterator shardIt, int shardIndex) { + AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); + assert filter != null; + float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); + CanMatchRequest.ShardLevelRequest shardRequest = new CanMatchRequest.ShardLevelRequest(request, shardIt.shardId(), shardIndex, + getNumShards(), filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), + shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); + // if we already received a search result we can inform the shard that it + // can return a null response if the request rewrites to match none rather + // than creating an empty response in the search thread pool. + // Note that, we have to disable this shortcut for queries that create a context (scroll and search context). + //shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null); + return shardRequest; + } + + private boolean checkMinimumVersion(GroupShardsIterator shardsIts) { + for (SearchShardIterator it : shardsIts) { + if (it.getTargetNodeIds().isEmpty() == false) { + boolean isCompatible = it.getTargetNodeIds().stream().anyMatch(nodeId -> { + Transport.Connection conn = getConnection(it.getClusterAlias(), nodeId); + return conn == null ? true : conn.getVersion().onOrAfter(request.minCompatibleShardNode()); + }); + if (isCompatible == false) { + return false; + } + } + } + return true; + } + + @Override + public void start() { + if (getNumShards() == 0) { + //no search shards to search on, bail with empty response + //(it happens with search across _all with no indices around and consistent with broadcast operations) + int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : + request.source().trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : + request.source().trackTotalHitsUpTo(); + // total hits is null in the response if the tracking of total hits is disabled + boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; + listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, 0, buildTookInMillis(), + ShardSearchFailure.EMPTY_ARRAY, clusters, null)); + return; + } + + try { + run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(this, "", e); + } + } + + + public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + // listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + } + + public final Transport.Connection getConnection(String clusterAlias, String nodeId) { + Transport.Connection conn = nodeIdToConnection.apply(clusterAlias, nodeId); + Version minVersion = request.minCompatibleShardNode(); + if (minVersion != null && conn != null && conn.getVersion().before(minVersion)) { + throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", minVersion); + } + return conn; + } + + private int getNumShards() { + return shardsIts.size(); + } + + long buildTookInMillis() { + return timeProvider.buildTookInMillis(); + } + + + private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { + private final FixedBitSet possibleMatches; + private final MinAndMax[] minAndMaxes; + private int numPossibleMatches; + + CanMatchSearchPhaseResults(int size) { + super(size); + possibleMatches = new FixedBitSet(size); + minAndMaxes = new MinAndMax[size]; + } + + @Override + void consumeResult(SearchService.CanMatchResponse result, Runnable next) { + try { + consumeResult(result.getShardIndex(), result.canMatch(), result.estimatedMinAndMax()); + } finally { + next.run(); + } + } + + @Override + boolean hasResult(int shardIndex) { + return false; // unneeded + } + + @Override + void consumeShardFailure(int shardIndex) { + // we have to carry over shard failures in order to account for them in the response. + consumeResult(shardIndex, true, null); + } + + synchronized void consumeResult(int shardIndex, boolean canMatch, MinAndMax minAndMax) { + if (canMatch) { + possibleMatches.set(shardIndex); + numPossibleMatches++; + } + minAndMaxes[shardIndex] = minAndMax; + } + + synchronized int getNumPossibleMatches() { + return numPossibleMatches; + } + + synchronized FixedBitSet getPossibleMatches() { + return possibleMatches; + } + + @Override + Stream getSuccessfulResults() { + return Stream.empty(); + } + } + + private GroupShardsIterator getIterator(CanMatchSearchPhaseResults results, + GroupShardsIterator shardsIts) { + int cardinality = results.getNumPossibleMatches(); + FixedBitSet possibleMatches = results.getPossibleMatches(); + if (cardinality == 0) { + // this is a special case where we have no hit but we need to get at least one search response in order + // to produce a valid search result with all the aggs etc. + // Since it's possible that some of the shards that we're skipping are + // unavailable, we would try to query the node that at least has some + // shards available in order to produce a valid search result. + int shardIndexToQuery = 0; + for (int i = 0; i < shardsIts.size(); i++) { + if (shardsIts.get(i).size() > 0) { + shardIndexToQuery = i; + break; + } + } + possibleMatches.set(shardIndexToQuery); + } + SearchSourceBuilder source = request.source(); + int i = 0; + for (SearchShardIterator iter : shardsIts) { + if (possibleMatches.get(i++)) { + iter.reset(); + } else { + iter.resetAndSkip(); + } + } + if (shouldSortShards(results.minAndMaxes) == false) { + return shardsIts; + } + FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); + return new GroupShardsIterator<>(sortShards(shardsIts, results.minAndMaxes, fieldSort.order())); + } + + private static List sortShards(GroupShardsIterator shardsIts, + MinAndMax[] minAndMaxes, + SortOrder order) { + return IntStream.range(0, shardsIts.size()) + .boxed() + .sorted(shardComparator(shardsIts, minAndMaxes, order)) + .map(shardsIts::get) + .collect(Collectors.toList()); + } + + private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { + Class clazz = null; + for (MinAndMax minAndMax : minAndMaxes) { + if (clazz == null) { + clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); + } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { + // we don't support sort values that mix different types (e.g.: long/double, numeric/keyword). + // TODO: we could fail the request because there is a high probability + // that the merging of topdocs will fail later for the same reason ? + return false; + } + } + return clazz != null; + } + + private static Comparator shardComparator(GroupShardsIterator shardsIts, + MinAndMax[] minAndMaxes, + SortOrder order) { + final Comparator comparator = Comparator.comparing( + index -> minAndMaxes[index], + forciblyCast(MinAndMax.getComparator(order)) + ); + + return comparator.thenComparing(index -> shardsIts.get(index)); + } + +} + diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java new file mode 100644 index 0000000000000..81f13102f604e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -0,0 +1,231 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.search; + +import org.elasticsearch.Version; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.Scroll; +import org.elasticsearch.search.SearchSortValuesAndFormats; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchContextId; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.transport.TransportRequest; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Node-level request used during can-match phase + */ +public class CanMatchRequest extends TransportRequest implements IndicesRequest { + + private final OriginalIndices originalIndices; + private final SearchSourceBuilder source; + private final List shards; + + public static class ShardLevelRequest implements Writeable { + private final String clusterAlias; + private final ShardId shardId; + private final int shardRequestIndex; + private final int numberOfShards; + private final SearchType searchType; + private final Scroll scroll; + private final float indexBoost; + private final Boolean requestCache; + private final long nowInMillis; + private final boolean allowPartialSearchResults; + + private final boolean canReturnNullResponseIfMatchNoDocs; + private final SearchSortValuesAndFormats bottomSortValues; + + private final AliasFilter aliasFilter; + private final ShardSearchContextId readerId; + private final TimeValue keepAlive; + + private final Version channelVersion; + + public ShardLevelRequest( + SearchRequest searchRequest, + ShardId shardId, + int shardRequestIndex, + int numberOfShards, + AliasFilter aliasFilter, + float indexBoost, + long nowInMillis, + @Nullable String clusterAlias, + ShardSearchContextId readerId, + TimeValue keepAlive) { + this(shardId, + shardRequestIndex, + numberOfShards, + searchRequest.searchType(), + searchRequest.requestCache(), + aliasFilter, + indexBoost, + searchRequest.allowPartialSearchResults(), + searchRequest.scroll(), + nowInMillis, + clusterAlias, + readerId, + keepAlive); + // If allowPartialSearchResults is unset (ie null), the cluster-level default should have been substituted + // at this stage. Any NPEs in the above are therefore an error in request preparation logic. + assert searchRequest.allowPartialSearchResults() != null; + } + + public ShardLevelRequest( ShardId shardId, + int shardRequestIndex, + int numberOfShards, + SearchType searchType, + Boolean requestCache, + AliasFilter aliasFilter, + float indexBoost, + boolean allowPartialSearchResults, + Scroll scroll, + long nowInMillis, + @Nullable String clusterAlias, + ShardSearchContextId readerId, + TimeValue keepAlive) { + this.shardId = shardId; + this.shardRequestIndex = shardRequestIndex; + this.numberOfShards = numberOfShards; + this.searchType = searchType; + this.requestCache = requestCache; + this.aliasFilter = aliasFilter; + this.indexBoost = indexBoost; + this.allowPartialSearchResults = allowPartialSearchResults; + this.scroll = scroll; + this.nowInMillis = nowInMillis; + this.clusterAlias = clusterAlias; + this.readerId = readerId; + this.keepAlive = keepAlive; + assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; + this.channelVersion = Version.CURRENT; + // TODO: remove the following two fields + this.canReturnNullResponseIfMatchNoDocs = false; + this.bottomSortValues = null; + } + + public ShardLevelRequest(StreamInput in) throws IOException { + // TODO: parent task super(in); + shardId = new ShardId(in); + searchType = SearchType.fromId(in.readByte()); + shardRequestIndex = in.getVersion().onOrAfter(Version.V_7_11_0) ? in.readVInt() : -1; + numberOfShards = in.readVInt(); + scroll = in.readOptionalWriteable(Scroll::new); + if (in.getVersion().before(Version.V_8_0_0)) { + // types no longer relevant so ignore + String[] types = in.readStringArray(); + if (types.length > 0) { + throw new IllegalStateException( + "types are no longer supported in search requests but found [" + Arrays.toString(types) + "]"); + } + } + aliasFilter = new AliasFilter(in); + indexBoost = in.readFloat(); + nowInMillis = in.readVLong(); + requestCache = in.readOptionalBoolean(); + clusterAlias = in.readOptionalString(); + allowPartialSearchResults = in.readBoolean(); + canReturnNullResponseIfMatchNoDocs = in.readBoolean(); + bottomSortValues = in.readOptionalWriteable(SearchSortValuesAndFormats::new); + readerId = in.readOptionalWriteable(ShardSearchContextId::new); + keepAlive = in.readOptionalTimeValue(); + assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; + channelVersion = Version.min(Version.readVersion(in), in.getVersion()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + out.writeByte(searchType.id()); + out.writeVInt(shardRequestIndex); + out.writeVInt(numberOfShards); + out.writeOptionalWriteable(scroll); + if (out.getVersion().before(Version.V_8_0_0)) { + // types not supported so send an empty array to previous versions + out.writeStringArray(Strings.EMPTY_ARRAY); + } + aliasFilter.writeTo(out); + out.writeFloat(indexBoost); + out.writeVLong(nowInMillis); + out.writeOptionalBoolean(requestCache); + out.writeOptionalString(clusterAlias); + out.writeBoolean(allowPartialSearchResults); + out.writeStringArray(Strings.EMPTY_ARRAY); + out.writeOptionalString(null); + out.writeBoolean(canReturnNullResponseIfMatchNoDocs); + out.writeOptionalWriteable(bottomSortValues); + out.writeOptionalWriteable(readerId); + out.writeOptionalTimeValue(keepAlive); + Version.writeVersion(channelVersion, out); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalWriteable(source); + OriginalIndices.writeOriginalIndices(originalIndices, out); + } + + public CanMatchRequest( + OriginalIndices originalIndices, + SearchRequest searchRequest, + List shards + ) { + this.originalIndices = originalIndices; + this.source = searchRequest.source(); + this.shards = new ArrayList<>(shards); + } + + public CanMatchRequest(StreamInput in) throws IOException { + source = in.readOptionalWriteable(SearchSourceBuilder::new); + originalIndices = OriginalIndices.readOriginalIndices(in); + shards = in.readList(ShardLevelRequest::new); + } + + @Override + public String[] indices() { + if (originalIndices == null) { + return null; + } + return originalIndices.indices(); + } + + @Override + public IndicesOptions indicesOptions() { + if (originalIndices == null) { + return null; + } + return originalIndices.indicesOptions(); + } + + public List createShardSearchRequests() { + return shards.stream().map(r -> new ShardSearchRequest( + originalIndices, r.shardId, r.shardRequestIndex, r.numberOfShards, r.searchType, + source, r.requestCache, r.aliasFilter, r.indexBoost, r.allowPartialSearchResults, r.scroll, + r.nowInMillis, r.clusterAlias, r.readerId, r.keepAlive + )).collect(Collectors.toList()); + } + +} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 83f0001972e81..3de90d727fdfb 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -28,4 +28,8 @@ protected SearchPhase(String name) { public String getName() { return name; } + + public void start() { + + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 41860c52174d4..3f0da22ff1306 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -73,6 +73,7 @@ public class SearchTransportService { public static final String FETCH_ID_SCROLL_ACTION_NAME = "indices:data/read/search[phase/fetch/id/scroll]"; public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; public static final String QUERY_CAN_MATCH_NAME = "indices:data/read/search[can_match]"; + public static final String QUERY_CAN_MATCH_NODE_NAME = "indices:data/read/search[can_match][n]"; private final TransportService transportService; private final NodeClient client; @@ -122,6 +123,12 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchService.CanMatchResponse::new)); } + public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final + ActionListener listener) { + transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchService.CanMatchResponse::new)); + } + public void sendClearAllScrollContexts(Transport.Connection connection, final ActionListener listener) { transportService.sendRequest(connection, CLEAR_SCROLL_CONTEXTS_ACTION_NAME, TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, (in) -> TransportResponse.Empty.INSTANCE)); @@ -364,6 +371,11 @@ public static void registerRequestHandler(TransportService transportService, Sea searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, true, SearchService.CanMatchResponse::new); + transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SAME, CanMatchRequest::new, + (request, channel, task) -> { + searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); + }); + TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NODE_NAME, true, CanMatchNodeResponse::new); } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 847be29fa4ddc..2cdca79d2ab46 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -768,14 +768,14 @@ static GroupShardsIterator mergeShardsIterators(List asyncSearchAction( + SearchPhase asyncSearchAction( SearchTask task, SearchRequest searchRequest, Executor executor, GroupShardsIterator shardIterators, SearchTimeProvider timeProvider, BiFunction connectionLookup, ClusterState clusterState, Map aliasFilter, Map concreteIndexBoosts, ActionListener listener, boolean preFilter, ThreadPool threadPool, SearchResponse.Clusters clusters); } - private AbstractSearchAsyncAction searchAsyncAction( + private SearchPhase searchAsyncAction( SearchTask task, SearchRequest searchRequest, Executor executor, @@ -790,10 +790,10 @@ private AbstractSearchAsyncAction searchAsyncAction ThreadPool threadPool, SearchResponse.Clusters clusters) { if (preFilter) { - return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup, + return new CanMatchPhase(logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators, timeProvider, clusterState, task, (iter) -> { - AbstractSearchAsyncAction action = searchAsyncAction( + SearchPhase action = searchAsyncAction( task, searchRequest, executor, @@ -807,6 +807,7 @@ private AbstractSearchAsyncAction searchAsyncAction false, threadPool, clusters); + assert action instanceof AbstractSearchAsyncAction; return new SearchPhase(action.getName()) { @Override public void run() { diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 175f028435ed6..aae70b9f41386 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -19,6 +19,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.search.CanMatchNodeResponse; +import org.elasticsearch.action.search.CanMatchRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; @@ -120,6 +122,7 @@ import org.elasticsearch.transport.TransportRequest; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -1223,6 +1226,18 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { + try { + List responses = new ArrayList<>(); + for (ShardSearchRequest shardSearchRequest : request.createShardSearchRequests()) { + responses.add(canMatch(shardSearchRequest)); + } + listener.onResponse(new CanMatchNodeResponse(responses)); + } catch (IOException e) { + listener.onFailure(e); + } + } + /** * This method uses a lightweight searcher without wrapping (i.e., not open a full reader on frozen indices) to rewrite the query * to check if the query can match any documents. This method can have false positives while if it returns {@code false} the query diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 81584252655a7..ce1aaa172d0dc 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -136,7 +136,7 @@ public ShardSearchRequest(ShardId shardId, aliasFilter, 1.0f, true, null, nowInMillis, null, null, null); } - private ShardSearchRequest(OriginalIndices originalIndices, + public ShardSearchRequest(OriginalIndices originalIndices, ShardId shardId, int shardRequestIndex, int numberOfShards, From 07e9f518e86a884542ae99e3f92ad8c7d525a9b9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 30 Sep 2021 16:04:03 +0200 Subject: [PATCH 02/28] happy path --- .../action/search/CanMatchPhase.java | 68 +++++++++++++++++-- .../action/search/CanMatchRequest.java | 25 ++++--- .../action/search/SearchTransportService.java | 4 +- .../action/search/TransportSearchAction.java | 3 +- 4 files changed, 80 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index e0a37ba721298..d400a08197ea3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -16,6 +16,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; @@ -35,6 +36,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -56,6 +59,12 @@ public class CanMatchPhase extends SearchPhase { private final Map shardItIndexMap; private final Map concreteIndexBoosts; private final Map aliasFilter; + private final SearchTask task; + private final Function, SearchPhase> phaseFactory; + + private final AtomicReferenceArray responses; + private final CanMatchSearchPhaseResults results; + private final AtomicInteger counter = new AtomicInteger(); public CanMatchPhase(Logger logger, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, @@ -76,7 +85,11 @@ public CanMatchPhase(Logger logger, SearchTransportService searchTransportServic this.timeProvider = timeProvider; this.concreteIndexBoosts = concreteIndexBoosts; this.aliasFilter = aliasFilter; + this.task = task; + this.phaseFactory = phaseFactory; this.shardItIndexMap = new HashMap<>(); + this.responses = new AtomicReferenceArray<>(shardsIts.size()); + results = new CanMatchSearchPhaseResults(shardsIts.size()); // we compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is @@ -125,23 +138,63 @@ public void run() throws IOException { private void runRound() { // create CanMatchRequests for the given round - Map> requests = new HashMap<>(); + Map, List> requests = new HashMap<>(); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); assert shardRoutings.skip() == false; -// assert shardItIndexMap.containsKey(shardRoutings); -// int shardIndex = shardItIndexMap.get(shardRoutings); + assert shardItIndexMap.containsKey(shardRoutings); SearchShardTarget target = shardRoutings.nextOrNull(); if (target != null) { - requests.computeIfAbsent(target.getNodeId(), t -> new ArrayList<>()).add(target); + requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), + t -> new ArrayList<>()).add(shardRoutings); + } else { + onOperation(shardItIndexMap.get(shardRoutings), null); } + } + for (Map.Entry, List> entry : requests.entrySet()) { + SearchShardIterator first = entry.getValue().get(0); + List shardLevelRequests = + entry.getValue().stream().map(ssi -> buildShardLevelRequest(ssi, shardItIndexMap.get(ssi))).collect(Collectors.toList()); + CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, + shardLevelRequests); + // TODO: use proper cluster alias + searchTransportService.sendCanMatch(getConnection(entry.getKey().v1(), entry.getKey().v2()), canMatchRequest, + task, new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchResponse) { + for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { + SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); + response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + results.consumeResult(response, () -> {}); + onOperation(response.getShardIndex(), response); + } + } - //performPhaseOnShard(shardIndex, shardRoutings, shardRoutings.nextOrNull()); + @Override + public void onFailure(Exception e) { + assert false : e; + } + } + ); } + } - for (Map.Entry> entry : requests.entrySet()) { - //new CanMatchRequest(getOriginalIndices(), request); + private void onOperation(int idx, SearchService.CanMatchResponse response) { + responses.set(idx, response); + if (counter.incrementAndGet() == responses.length()) { + finishHim(); + } + } + + private void finishHim() { + try { + phaseFactory.apply(getIterator(results, shardsIts)).run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(this, "", e); } } @@ -204,6 +257,7 @@ public void start() { public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + assert false : cause; // listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 81f13102f604e..1f3cdd64c4ec2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -45,6 +45,11 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest public static class ShardLevelRequest implements Writeable { private final String clusterAlias; private final ShardId shardId; + + public int getShardRequestIndex() { + return shardRequestIndex; + } + private final int shardRequestIndex; private final int numberOfShards; private final SearchType searchType; @@ -129,7 +134,7 @@ public ShardLevelRequest(StreamInput in) throws IOException { // TODO: parent task super(in); shardId = new ShardId(in); searchType = SearchType.fromId(in.readByte()); - shardRequestIndex = in.getVersion().onOrAfter(Version.V_7_11_0) ? in.readVInt() : -1; + shardRequestIndex = in.readVInt(); numberOfShards = in.readVInt(); scroll = in.readOptionalWriteable(Scroll::new); if (in.getVersion().before(Version.V_8_0_0)) { @@ -171,8 +176,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalBoolean(requestCache); out.writeOptionalString(clusterAlias); out.writeBoolean(allowPartialSearchResults); - out.writeStringArray(Strings.EMPTY_ARRAY); - out.writeOptionalString(null); out.writeBoolean(canReturnNullResponseIfMatchNoDocs); out.writeOptionalWriteable(bottomSortValues); out.writeOptionalWriteable(readerId); @@ -181,13 +184,6 @@ public void writeTo(StreamOutput out) throws IOException { } } - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeOptionalWriteable(source); - OriginalIndices.writeOriginalIndices(originalIndices, out); - } - public CanMatchRequest( OriginalIndices originalIndices, SearchRequest searchRequest, @@ -199,11 +195,20 @@ public CanMatchRequest( } public CanMatchRequest(StreamInput in) throws IOException { + super(in); source = in.readOptionalWriteable(SearchSourceBuilder::new); originalIndices = OriginalIndices.readOriginalIndices(in); shards = in.readList(ShardLevelRequest::new); } + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalWriteable(source); + OriginalIndices.writeOriginalIndices(originalIndices, out); + out.writeList(shards); + } + @Override public String[] indices() { if (originalIndices == null) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 3f0da22ff1306..d6bd627967130 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -124,9 +124,9 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque } public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final - ActionListener listener) { + ActionListener listener) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, - TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchService.CanMatchResponse::new)); + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); } public void sendClearAllScrollContexts(Transport.Connection connection, final ActionListener listener) { diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 2cdca79d2ab46..ab1f50c0f9a5a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -688,7 +688,8 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea BiFunction connectionLookup = buildConnectionLookup(searchRequest.getLocalClusterAlias(), nodes::get, remoteConnections, searchTransportService::getConnection); final Executor asyncSearchExecutor = asyncSearchExecutor(concreteLocalIndices); - final boolean preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, concreteLocalIndices, + final boolean preFilterSearchShards = + shouldPreFilterSearchShards(clusterState, searchRequest, concreteLocalIndices, localShardIterators.size() + remoteShardIterators.size()); searchAsyncActionProvider.asyncSearchAction( task, searchRequest, asyncSearchExecutor, shardIterators, timeProvider, connectionLookup, clusterState, From 53d1071aa2e3f427d05b4ae5d35177a82226ad5d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Oct 2021 14:13:52 +0200 Subject: [PATCH 03/28] more work --- .../action/search/CanMatchNodeResponse.java | 14 +- .../action/search/CanMatchPhase.java | 160 +++++++++++++----- .../action/search/CanMatchRequest.java | 8 +- .../action/search/SearchTransportService.java | 51 +++++- .../action/search/TransportSearchAction.java | 2 +- .../elasticsearch/search/SearchService.java | 21 ++- .../transport/TransportActionProxy.java | 14 +- 7 files changed, 203 insertions(+), 67 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java index c79c7b154f9c0..9eee993f8c3ca 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.search; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.SearchService; @@ -19,23 +20,30 @@ public class CanMatchNodeResponse extends TransportResponse { private final List responses; + private final List failures; public CanMatchNodeResponse(StreamInput in) throws IOException { super(in); - responses = in.readList(SearchService.CanMatchResponse::new); + responses = in.readList(i -> i.readOptionalWriteable(SearchService.CanMatchResponse::new)); + failures = in.readList(StreamInput::readException); } - public CanMatchNodeResponse(List responses) { + public CanMatchNodeResponse(List responses, List failures) { this.responses = responses; + this.failures = failures; } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeList(responses); + out.writeCollection(responses, StreamOutput::writeOptionalWriteable); + out.writeCollection(failures, StreamOutput::writeException); } public List getResponses() { return responses; } + public List getFailures() { + return failures; + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index d400a08197ea3..a01de37fbf2af 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -12,10 +12,12 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.search.SearchService; @@ -62,9 +64,10 @@ public class CanMatchPhase extends SearchPhase { private final SearchTask task; private final Function, SearchPhase> phaseFactory; - private final AtomicReferenceArray responses; private final CanMatchSearchPhaseResults results; - private final AtomicInteger counter = new AtomicInteger(); + private final SetOnce> shardFailures = new SetOnce<>(); + private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; + public CanMatchPhase(Logger logger, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, @@ -87,8 +90,8 @@ public CanMatchPhase(Logger logger, SearchTransportService searchTransportServic this.aliasFilter = aliasFilter; this.task = task; this.phaseFactory = phaseFactory; + this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; this.shardItIndexMap = new HashMap<>(); - this.responses = new AtomicReferenceArray<>(shardsIts.size()); results = new CanMatchSearchPhaseResults(shardsIts.size()); // we compute the shard index based on the natural order of the shards @@ -132,58 +135,110 @@ public void run() throws IOException { request.minCompatibleShardNode()); } } - runRound(); + new Round(shardsIts).run(); } } - private void runRound() { - // create CanMatchRequests for the given round - Map, List> requests = new HashMap<>(); - for (int i = 0; i < shardsIts.size(); i++) { - final SearchShardIterator shardRoutings = shardsIts.get(i); - assert shardRoutings.skip() == false; - assert shardItIndexMap.containsKey(shardRoutings); - SearchShardTarget target = shardRoutings.nextOrNull(); - if (target != null) { - requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), - t -> new ArrayList<>()).add(shardRoutings); - } else { - onOperation(shardItIndexMap.get(shardRoutings), null); - } + class Round implements Runnable { + private final GroupShardsIterator shards; + private final AtomicInteger counter = new AtomicInteger(); + private final AtomicReferenceArray responses; + + Round(GroupShardsIterator shards) { + this.shards = shards; + this.responses = new AtomicReferenceArray<>(shardsIts.size()); } - for (Map.Entry, List> entry : requests.entrySet()) { - SearchShardIterator first = entry.getValue().get(0); - List shardLevelRequests = - entry.getValue().stream().map(ssi -> buildShardLevelRequest(ssi, shardItIndexMap.get(ssi))).collect(Collectors.toList()); - CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, - shardLevelRequests); - // TODO: use proper cluster alias - searchTransportService.sendCanMatch(getConnection(entry.getKey().v1(), entry.getKey().v2()), canMatchRequest, - task, new ActionListener<>() { - @Override - public void onResponse(CanMatchNodeResponse canMatchResponse) { - for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); - response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); - results.consumeResult(response, () -> {}); - onOperation(response.getShardIndex(), response); + + @Override + public void run() { + // create CanMatchRequests for the given round + Map, List> requests = new HashMap<>(); + for (int i = 0; i < shards.size(); i++) { + final SearchShardIterator shardRoutings = shards.get(i); + assert shardRoutings.skip() == false; + assert shardItIndexMap.containsKey(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), + t -> new ArrayList<>()).add(shardRoutings); + } else { + results.consumeShardFailure(shardItIndexMap.get(shardRoutings)); + onOperation(shardItIndexMap.get(shardRoutings), null); + } + } + + for (Map.Entry, List> entry : requests.entrySet()) { + SearchShardIterator first = entry.getValue().get(0); + List shardLevelRequests = + entry.getValue().stream().map(ssi -> buildShardLevelRequest(ssi, shardItIndexMap.get(ssi))) + .collect(Collectors.toList()); + CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, + shardLevelRequests); + searchTransportService.sendCanMatch(getConnection(entry.getKey().v1(), entry.getKey().v2()), canMatchRequest, + task, new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchResponse) { + assert shardLevelRequests.size() == canMatchResponse.getResponses().size() && + shardLevelRequests.size() == canMatchResponse.getFailures().size(); + for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { + SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); + if (response != null) { + response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + results.consumeResult(response, () -> { + }); + onOperation(response.getShardIndex(), response); + } + } + for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { + Exception failure = canMatchResponse.getFailures().get(i); + if (failure != null) { + results.consumeShardFailure(i); + onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + } + } } - } - @Override - public void onFailure(Exception e) { - assert false : e; + @Override + public void onFailure(Exception e) { + for (CanMatchRequest.ShardLevelRequest shardLevelRequest : shardLevelRequests) { + onOperationFailed(shardLevelRequest.getShardRequestIndex(), e); + } + } } - } - ); + ); + } } - } - private void onOperation(int idx, SearchService.CanMatchResponse response) { - responses.set(idx, response); - if (counter.incrementAndGet() == responses.length()) { - finishHim(); + private void onOperation(int idx, SearchService.CanMatchResponse response) { + responses.set(idx, response); + if (counter.incrementAndGet() == responses.length()) { + finishPhase(); + } + } + + private void onOperationFailed(int idx, Exception e) { + responses.set(idx, e); + if (counter.incrementAndGet() == shards.size()) { + finishPhase(); + } + } + + private void finishPhase() { + List remainingShards = new ArrayList<>(); + for (SearchShardIterator ssi : shards) { + int shardIndex = shardItIndexMap.get(ssi); + Object resp = responses.get(shardIndex); + if (resp instanceof Exception) { + // do something meaningful + remainingShards.add(ssi); + } + } + if (remainingShards.isEmpty()) { + finishHim(); + } else { + new Round(new GroupShardsIterator<>(remainingShards)).run(); + } } } @@ -198,6 +253,19 @@ private void finishHim() { } } + private ShardSearchFailure[] buildShardFailures() { + AtomicArray shardFailures = this.shardFailures.get(); + if (shardFailures == null) { + return ShardSearchFailure.EMPTY_ARRAY; + } + List entries = shardFailures.asList(); + ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; + for (int i = 0; i < failures.length; i++) { + failures[i] = entries.get(i); + } + return failures; + } + private static final float DEFAULT_INDEX_BOOST = 1.0f; public final CanMatchRequest.ShardLevelRequest buildShardLevelRequest(SearchShardIterator shardIt, int shardIndex) { @@ -258,7 +326,7 @@ public void start() { public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { assert false : cause; - // listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); } public final Transport.Connection getConnection(String clusterAlias, String nodeId) { diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 1f3cdd64c4ec2..dbf22631e35cf 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -226,11 +226,15 @@ public IndicesOptions indicesOptions() { } public List createShardSearchRequests() { - return shards.stream().map(r -> new ShardSearchRequest( + return shards.stream().map(this::createShardSearchRequest).collect(Collectors.toList()); + } + + public ShardSearchRequest createShardSearchRequest(ShardLevelRequest r) { + return new ShardSearchRequest( originalIndices, r.shardId, r.shardRequestIndex, r.numberOfShards, r.searchType, source, r.requestCache, r.aliasFilter, r.indexBoost, r.allowPartialSearchResults, r.scroll, r.nowInMillis, r.clusterAlias, r.readerId, r.keepAlive - )).collect(Collectors.toList()); + ); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index d6bd627967130..893036c3e9c4b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.search; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.IndicesRequest; @@ -51,9 +52,13 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; /** @@ -125,8 +130,50 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, - TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); + if (connection.getVersion().onOrAfter(Version.V_8_0_0)) { + transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); + } else { + List shardSearchRequests = request.createShardSearchRequests(); + AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); + AtomicInteger counter = new AtomicInteger(); + for (int i = 0; i < shardSearchRequests.size(); i++) { + ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); + // TODO: do we need to set parent task etc on this synthetic ShardSearchRequest + int finalI = i; + sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchService.CanMatchResponse response) { + results.set(finalI, response); + maybeFinish(); + } + + @Override + public void onFailure(Exception e) { + results.set(finalI, e); + maybeFinish(); + } + + private void maybeFinish() { + if (counter.incrementAndGet() == shardSearchRequests.size()) { + List responses = new ArrayList<>(shardSearchRequests.size()); + List failures = new ArrayList<>(shardSearchRequests.size()); + for (int i = 0; i < results.length(); i++) { + Object o = results.get(i); + if (o instanceof SearchService.CanMatchResponse) { + responses.set(i, (SearchService.CanMatchResponse) o); + } else { + assert o instanceof Exception; + failures.set(i, (Exception) o); + } + } + CanMatchNodeResponse response = new CanMatchNodeResponse(responses, failures); + listener.onResponse(response); + } + } + }); + } + } } public void sendClearAllScrollContexts(Transport.Connection connection, final ActionListener listener) { diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index ab1f50c0f9a5a..0e65b7cd57e50 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -744,7 +744,7 @@ static boolean shouldPreFilterSearchShards(ClusterState clusterState, && (hasReadOnlyIndices(indices, clusterState) || hasPrimaryFieldSort(source))) { preFilterShardSize = 1; } else if (preFilterShardSize == null) { - preFilterShardSize = SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE; + preFilterShardSize = 1; } return searchRequest.searchType() == QUERY_THEN_FETCH // we can't do this for DFS it needs to fan out to all shards all the time && (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source)) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index aae70b9f41386..1fbf78c8013ec 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -15,6 +15,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.TopDocs; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; @@ -1227,15 +1228,21 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { - try { - List responses = new ArrayList<>(); - for (ShardSearchRequest shardSearchRequest : request.createShardSearchRequests()) { - responses.add(canMatch(shardSearchRequest)); + List responses = new ArrayList<>(); + List failures = new ArrayList<>(); + List shardSearchRequests = request.createShardSearchRequests(); + for (int i = 0; i < shardSearchRequests.size(); i++) { + ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); + CanMatchResponse canMatchResponse; + try { + canMatchResponse = canMatch(shardSearchRequest); + responses.set(i, canMatchResponse); + } catch (Exception e) { + failures.set(i, e); } - listener.onResponse(new CanMatchNodeResponse(responses)); - } catch (IOException e) { - listener.onFailure(e); + } + listener.onResponse(new CanMatchNodeResponse(responses, failures)); } /** diff --git a/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index a7ae4537b1f14..36ee37300fe86 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -63,6 +63,13 @@ private boolean assertConsistentTaskType(Task proxyTask, TransportRequest wrappe } } + interface ResendingContext { + String getAction(); + TransportRequest wrappedRequest(); + DiscoveryNode targetNode(); + TransportRequestOptions options(); + } + private static class ProxyResponseHandler implements TransportResponseHandler { private final Writeable.Reader reader; @@ -159,12 +166,7 @@ public static void registerProxyActionWithDynamicResponseType(TransportService s */ public static void registerProxyAction(TransportService service, String action, boolean cancellable, Writeable.Reader reader) { - RequestHandlerRegistry requestHandler = service.getRequestHandler(action); - service.registerRequestHandler(getProxyAction(action), ThreadPool.Names.SAME, true, false, - in -> cancellable ? - new CancellableProxyRequest<>(in, requestHandler::newRequest) : - new ProxyRequest<>(in, requestHandler::newRequest), - new ProxyRequestHandler<>(service, action, request -> reader)); + registerProxyActionWithDynamicResponseType(service, action, cancellable, request -> reader); } private static final String PROXY_ACTION_PREFIX = "internal:transport/proxy/"; From 2263bbdcb4cfa70db25555c74c92c429bd0edb91 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Oct 2021 16:20:45 +0200 Subject: [PATCH 04/28] more edits --- .../action/search/CanMatchPhase.java | 200 ++++++++++++------ .../action/search/CanMatchRequest.java | 135 +++++------- 2 files changed, 180 insertions(+), 155 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index a01de37fbf2af..c75fc55e36663 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -12,13 +12,12 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.query.CoordinatorRewriteContext; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; @@ -26,6 +25,7 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.MinAndMax; import org.elasticsearch.search.sort.SortOrder; @@ -33,10 +33,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; @@ -65,7 +67,6 @@ public class CanMatchPhase extends SearchPhase { private final Function, SearchPhase> phaseFactory; private final CanMatchSearchPhaseResults results; - private final SetOnce> shardFailures = new SetOnce<>(); private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; @@ -108,26 +109,7 @@ public CanMatchPhase(Logger logger, SearchTransportService searchTransportServic @Override public void run() throws IOException { if (shardsIts.size() > 0) { - assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (request.allowPartialSearchResults() == false) { - final StringBuilder missingShards = new StringBuilder(); - // Fail-fast verification of all shards being available - for (int index = 0; index < shardsIts.size(); index++) { - final SearchShardIterator shardRoutings = shardsIts.get(index); - if (shardRoutings.size() == 0) { - if(missingShards.length() > 0){ - missingShards.append(", "); - } - missingShards.append(shardRoutings.shardId()); - } - } - if (missingShards.length() > 0) { - //Status red - shard is missing all copies and would produce partial results for an index search - final String msg = "Search rejected due to missing shards ["+ missingShards + - "]. Consider using `allow_partial_search_results` setting to bypass this error."; - throw new SearchPhaseExecutionException(getName(), msg, null, ShardSearchFailure.EMPTY_ARRAY); - } - } + checkNoMissingShards(); Version version = request.minCompatibleShardNode(); if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { if (checkMinimumVersion(shardsIts) == false) { @@ -135,8 +117,90 @@ public void run() throws IOException { request.minCompatibleShardNode()); } } - new Round(shardsIts).run(); + + runCoordinationPhase(); + } + } + + private void runCoordinationPhase() { + final List matchedShardLevelRequests = new ArrayList<>(); + for (SearchShardIterator searchShardIterator : shardsIts) { + final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, + Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), + timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); + List shardLevelRequests = canMatchRequest.getShardLevelRequests(); + List shardSearchRequests = canMatchRequest.createShardSearchRequests(); + for (int i = 0; i < shardSearchRequests.size(); i++) { + ShardSearchRequest request = shardSearchRequests.get(i); + + CoordinatorRewriteContext coordinatorRewriteContext = + coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); + if (coordinatorRewriteContext != null) { + boolean canMatch = true; + try { + canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); + } catch (Exception e) { + // ignore + // treat as if shard is still a potential match + } + + if (canMatch) { + matchedShardLevelRequests.add(searchShardIterator); + } else { + SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); + result.setShardIndex(request.shardRequestIndex()); + results.consumeResult(result, () -> {}); + } + } + } + } + + if (matchedShardLevelRequests.isEmpty() == false) { + new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).run(); + } else { + finishHim(); + } + } + + private void checkNoMissingShards() { + assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (request.allowPartialSearchResults() == false) { + final StringBuilder missingShards = new StringBuilder(); + // Fail-fast verification of all shards being available + for (int index = 0; index < shardsIts.size(); index++) { + final SearchShardIterator shardRoutings = shardsIts.get(index); + if (shardRoutings.size() == 0) { + if (missingShards.length() > 0) { + missingShards.append(", "); + } + missingShards.append(shardRoutings.shardId()); + } + } + if (missingShards.length() > 0) { + //Status red - shard is missing all copies and would produce partial results for an index search + final String msg = "Search rejected due to missing shards ["+ missingShards + + "]. Consider using `allow_partial_search_results` setting to bypass this error."; + throw new SearchPhaseExecutionException(getName(), msg, null, ShardSearchFailure.EMPTY_ARRAY); + } + } + } + + private Map, List> groupByNode(GroupShardsIterator shards) { + Map, List> requests = new HashMap<>(); + for (int i = 0; i < shards.size(); i++) { + final SearchShardIterator shardRoutings = shards.get(i); + assert shardRoutings.skip() == false; + assert shardItIndexMap.containsKey(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), + t -> new ArrayList<>()).add(shardRoutings); + } else { + requests.computeIfAbsent(Tuple.tuple(null, null), + t -> new ArrayList<>()).add(shardRoutings); + } } + return requests; } class Round implements Runnable { @@ -149,32 +213,35 @@ class Round implements Runnable { this.responses = new AtomicReferenceArray<>(shardsIts.size()); } + public void start() { + try { + run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(CanMatchPhase.this, "", e); + } + } + @Override public void run() { - // create CanMatchRequests for the given round - Map, List> requests = new HashMap<>(); - for (int i = 0; i < shards.size(); i++) { - final SearchShardIterator shardRoutings = shards.get(i); - assert shardRoutings.skip() == false; - assert shardItIndexMap.containsKey(shardRoutings); - SearchShardTarget target = shardRoutings.nextOrNull(); - if (target != null) { - requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), - t -> new ArrayList<>()).add(shardRoutings); - } else { - results.consumeShardFailure(shardItIndexMap.get(shardRoutings)); - onOperation(shardItIndexMap.get(shardRoutings), null); - } - } + final Map, List> requests = groupByNode(shards); for (Map.Entry, List> entry : requests.entrySet()) { - SearchShardIterator first = entry.getValue().get(0); - List shardLevelRequests = - entry.getValue().stream().map(ssi -> buildShardLevelRequest(ssi, shardItIndexMap.get(ssi))) - .collect(Collectors.toList()); - CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, - shardLevelRequests); + CanMatchRequest canMatchRequest = createCanMatchRequest(entry); + List shardLevelRequests = canMatchRequest.getShardLevelRequests(); + + if (entry.getKey().v2() == null) { + // no target node + for (CanMatchRequest.ShardLevelRequest shardLevelRequest : shardLevelRequests) { + results.consumeShardFailure(shardLevelRequest.getShardRequestIndex()); + onOperation(shardLevelRequest.getShardRequestIndex(), null); + } + continue; + } + searchTransportService.sendCanMatch(getConnection(entry.getKey().v1(), entry.getKey().v2()), canMatchRequest, task, new ActionListener<>() { @Override @@ -237,11 +304,24 @@ private void finishPhase() { if (remainingShards.isEmpty()) { finishHim(); } else { - new Round(new GroupShardsIterator<>(remainingShards)).run(); + // trigger another round + new Round(new GroupShardsIterator<>(remainingShards)).start(); } } } + private CanMatchRequest createCanMatchRequest(Map.Entry, List> entry) { + final SearchShardIterator first = entry.getValue().get(0); + final List shardLevelRequests = + entry.getValue().stream().map(this::buildShardLevelRequest) + .collect(Collectors.toCollection(ArrayList::new)); + assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); + assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); + final CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, + shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); + return canMatchRequest; + } + private void finishHim() { try { phaseFactory.apply(getIterator(results, shardsIts)).run(); @@ -253,33 +333,14 @@ private void finishHim() { } } - private ShardSearchFailure[] buildShardFailures() { - AtomicArray shardFailures = this.shardFailures.get(); - if (shardFailures == null) { - return ShardSearchFailure.EMPTY_ARRAY; - } - List entries = shardFailures.asList(); - ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; - for (int i = 0; i < failures.length; i++) { - failures[i] = entries.get(i); - } - return failures; - } - private static final float DEFAULT_INDEX_BOOST = 1.0f; - public final CanMatchRequest.ShardLevelRequest buildShardLevelRequest(SearchShardIterator shardIt, int shardIndex) { + public final CanMatchRequest.ShardLevelRequest buildShardLevelRequest(SearchShardIterator shardIt) { AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - CanMatchRequest.ShardLevelRequest shardRequest = new CanMatchRequest.ShardLevelRequest(request, shardIt.shardId(), shardIndex, - getNumShards(), filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), - shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); - // if we already received a search result we can inform the shard that it - // can return a null response if the request rewrites to match none rather - // than creating an empty response in the search thread pool. - // Note that, we have to disable this shortcut for queries that create a context (scroll and search context). - //shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null); + CanMatchRequest.ShardLevelRequest shardRequest = new CanMatchRequest.ShardLevelRequest(shardIt.shardId(), + shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); return shardRequest; } @@ -325,8 +386,7 @@ public void start() { public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { - assert false : cause; - listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } public final Transport.Connection getConnection(String clusterAlias, String nodeId) { diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index dbf22631e35cf..8bc058e72c70a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -20,7 +20,6 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.Scroll; -import org.elasticsearch.search.SearchSortValuesAndFormats; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchContextId; @@ -41,9 +40,16 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest private final OriginalIndices originalIndices; private final SearchSourceBuilder source; private final List shards; + private final SearchType searchType; + private final Boolean requestCache; + private final boolean allowPartialSearchResults; + private final Scroll scroll; + private final int numberOfShards; + private final long nowInMillis; + @Nullable + private final String clusterAlias; public static class ShardLevelRequest implements Writeable { - private final String clusterAlias; private final ShardId shardId; public int getShardRequestIndex() { @@ -51,16 +57,7 @@ public int getShardRequestIndex() { } private final int shardRequestIndex; - private final int numberOfShards; - private final SearchType searchType; - private final Scroll scroll; private final float indexBoost; - private final Boolean requestCache; - private final long nowInMillis; - private final boolean allowPartialSearchResults; - - private final boolean canReturnNullResponseIfMatchNoDocs; - private final SearchSortValuesAndFormats bottomSortValues; private final AliasFilter aliasFilter; private final ShardSearchContextId readerId; @@ -68,75 +65,25 @@ public int getShardRequestIndex() { private final Version channelVersion; - public ShardLevelRequest( - SearchRequest searchRequest, - ShardId shardId, - int shardRequestIndex, - int numberOfShards, - AliasFilter aliasFilter, - float indexBoost, - long nowInMillis, - @Nullable String clusterAlias, - ShardSearchContextId readerId, - TimeValue keepAlive) { - this(shardId, - shardRequestIndex, - numberOfShards, - searchRequest.searchType(), - searchRequest.requestCache(), - aliasFilter, - indexBoost, - searchRequest.allowPartialSearchResults(), - searchRequest.scroll(), - nowInMillis, - clusterAlias, - readerId, - keepAlive); - // If allowPartialSearchResults is unset (ie null), the cluster-level default should have been substituted - // at this stage. Any NPEs in the above are therefore an error in request preparation logic. - assert searchRequest.allowPartialSearchResults() != null; - } - public ShardLevelRequest( ShardId shardId, int shardRequestIndex, - int numberOfShards, - SearchType searchType, - Boolean requestCache, AliasFilter aliasFilter, float indexBoost, - boolean allowPartialSearchResults, - Scroll scroll, - long nowInMillis, - @Nullable String clusterAlias, ShardSearchContextId readerId, TimeValue keepAlive) { this.shardId = shardId; this.shardRequestIndex = shardRequestIndex; - this.numberOfShards = numberOfShards; - this.searchType = searchType; - this.requestCache = requestCache; this.aliasFilter = aliasFilter; this.indexBoost = indexBoost; - this.allowPartialSearchResults = allowPartialSearchResults; - this.scroll = scroll; - this.nowInMillis = nowInMillis; - this.clusterAlias = clusterAlias; this.readerId = readerId; this.keepAlive = keepAlive; assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; this.channelVersion = Version.CURRENT; - // TODO: remove the following two fields - this.canReturnNullResponseIfMatchNoDocs = false; - this.bottomSortValues = null; } public ShardLevelRequest(StreamInput in) throws IOException { - // TODO: parent task super(in); shardId = new ShardId(in); - searchType = SearchType.fromId(in.readByte()); shardRequestIndex = in.readVInt(); - numberOfShards = in.readVInt(); - scroll = in.readOptionalWriteable(Scroll::new); if (in.getVersion().before(Version.V_8_0_0)) { // types no longer relevant so ignore String[] types = in.readStringArray(); @@ -147,12 +94,6 @@ public ShardLevelRequest(StreamInput in) throws IOException { } aliasFilter = new AliasFilter(in); indexBoost = in.readFloat(); - nowInMillis = in.readVLong(); - requestCache = in.readOptionalBoolean(); - clusterAlias = in.readOptionalString(); - allowPartialSearchResults = in.readBoolean(); - canReturnNullResponseIfMatchNoDocs = in.readBoolean(); - bottomSortValues = in.readOptionalWriteable(SearchSortValuesAndFormats::new); readerId = in.readOptionalWriteable(ShardSearchContextId::new); keepAlive = in.readOptionalTimeValue(); assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; @@ -162,22 +103,13 @@ public ShardLevelRequest(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); - out.writeByte(searchType.id()); out.writeVInt(shardRequestIndex); - out.writeVInt(numberOfShards); - out.writeOptionalWriteable(scroll); if (out.getVersion().before(Version.V_8_0_0)) { // types not supported so send an empty array to previous versions out.writeStringArray(Strings.EMPTY_ARRAY); } aliasFilter.writeTo(out); out.writeFloat(indexBoost); - out.writeVLong(nowInMillis); - out.writeOptionalBoolean(requestCache); - out.writeOptionalString(clusterAlias); - out.writeBoolean(allowPartialSearchResults); - out.writeBoolean(canReturnNullResponseIfMatchNoDocs); - out.writeOptionalWriteable(bottomSortValues); out.writeOptionalWriteable(readerId); out.writeOptionalTimeValue(keepAlive); Version.writeVersion(channelVersion, out); @@ -187,17 +119,37 @@ public void writeTo(StreamOutput out) throws IOException { public CanMatchRequest( OriginalIndices originalIndices, SearchRequest searchRequest, - List shards - ) { - this.originalIndices = originalIndices; - this.source = searchRequest.source(); - this.shards = new ArrayList<>(shards); + List shards, + int numberOfShards, + long nowInMillis, + @Nullable String clusterAlias + ) { + this.originalIndices = originalIndices; + this.source = searchRequest.source(); + this.shards = new ArrayList<>(shards); + this.searchType = searchRequest.searchType(); + this.requestCache = searchRequest.requestCache(); + // If allowPartialSearchResults is unset (ie null), the cluster-level default should have been substituted + // at this stage. Any NPEs in the above are therefore an error in request preparation logic. + assert searchRequest.allowPartialSearchResults() != null; + this.allowPartialSearchResults = searchRequest.allowPartialSearchResults(); + this.scroll = searchRequest.scroll(); + this.numberOfShards = numberOfShards; + this.nowInMillis = nowInMillis; + this.clusterAlias = clusterAlias; } public CanMatchRequest(StreamInput in) throws IOException { super(in); source = in.readOptionalWriteable(SearchSourceBuilder::new); originalIndices = OriginalIndices.readOriginalIndices(in); + searchType = SearchType.fromId(in.readByte()); + scroll = in.readOptionalWriteable(Scroll::new); + requestCache = in.readOptionalBoolean(); + allowPartialSearchResults = in.readBoolean(); + numberOfShards = in.readVInt(); + nowInMillis = in.readVLong(); + clusterAlias = in.readOptionalString(); shards = in.readList(ShardLevelRequest::new); } @@ -206,6 +158,13 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeOptionalWriteable(source); OriginalIndices.writeOriginalIndices(originalIndices, out); + out.writeByte(searchType.id()); + out.writeOptionalWriteable(scroll); + out.writeOptionalBoolean(requestCache); + out.writeBoolean(allowPartialSearchResults); + out.writeVInt(numberOfShards); + out.writeVLong(nowInMillis); + out.writeOptionalString(clusterAlias); out.writeList(shards); } @@ -225,16 +184,22 @@ public IndicesOptions indicesOptions() { return originalIndices.indicesOptions(); } + public List getShardLevelRequests() { + return shards; + } + public List createShardSearchRequests() { return shards.stream().map(this::createShardSearchRequest).collect(Collectors.toList()); } public ShardSearchRequest createShardSearchRequest(ShardLevelRequest r) { - return new ShardSearchRequest( - originalIndices, r.shardId, r.shardRequestIndex, r.numberOfShards, r.searchType, - source, r.requestCache, r.aliasFilter, r.indexBoost, r.allowPartialSearchResults, r.scroll, - r.nowInMillis, r.clusterAlias, r.readerId, r.keepAlive + ShardSearchRequest shardSearchRequest = new ShardSearchRequest( + originalIndices, r.shardId, r.shardRequestIndex, numberOfShards, searchType, + source, requestCache, r.aliasFilter, r.indexBoost, allowPartialSearchResults, scroll, + nowInMillis, clusterAlias, r.readerId, r.keepAlive ); + shardSearchRequest.setParentTask(getParentTask()); + return shardSearchRequest; } } From 66d2095243548bfaa275810b58cc9eecf1f5e8e6 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 5 Oct 2021 11:46:29 +0200 Subject: [PATCH 05/28] more stuff --- .../action/search/CanMatchPhase.java | 28 +++++++++---------- .../action/search/CanMatchRequest.java | 14 ++++++++++ .../action/search/SearchTransportService.java | 12 ++++---- .../action/search/TransportSearchAction.java | 11 ++------ .../elasticsearch/search/SearchService.java | 8 +++--- 5 files changed, 41 insertions(+), 32 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index c75fc55e36663..35f52bc9bb8be 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -128,35 +128,32 @@ private void runCoordinationPhase() { final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); - List shardLevelRequests = canMatchRequest.getShardLevelRequests(); List shardSearchRequests = canMatchRequest.createShardSearchRequests(); for (int i = 0; i < shardSearchRequests.size(); i++) { ShardSearchRequest request = shardSearchRequests.get(i); + boolean canMatch = true; CoordinatorRewriteContext coordinatorRewriteContext = coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); if (coordinatorRewriteContext != null) { - boolean canMatch = true; try { canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); } catch (Exception e) { - // ignore // treat as if shard is still a potential match } - - if (canMatch) { - matchedShardLevelRequests.add(searchShardIterator); - } else { - SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); - result.setShardIndex(request.shardRequestIndex()); - results.consumeResult(result, () -> {}); - } + } + if (canMatch) { + matchedShardLevelRequests.add(searchShardIterator); + } else { + SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); + result.setShardIndex(request.shardRequestIndex()); + results.consumeResult(result, () -> {}); } } } if (matchedShardLevelRequests.isEmpty() == false) { - new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).run(); + new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).start(); } else { finishHim(); } @@ -246,8 +243,7 @@ public void run() { task, new ActionListener<>() { @Override public void onResponse(CanMatchNodeResponse canMatchResponse) { - assert shardLevelRequests.size() == canMatchResponse.getResponses().size() && - shardLevelRequests.size() == canMatchResponse.getFailures().size(); + logger.info("Happy response"); for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); if (response != null) { @@ -268,7 +264,9 @@ public void onResponse(CanMatchNodeResponse canMatchResponse) { @Override public void onFailure(Exception e) { + logger.error(e); for (CanMatchRequest.ShardLevelRequest shardLevelRequest : shardLevelRequests) { + results.consumeShardFailure(shardLevelRequest.getShardRequestIndex()); onOperationFailed(shardLevelRequest.getShardRequestIndex(), e); } } @@ -324,7 +322,7 @@ private CanMatchRequest createCanMatchRequest(Map.Entry, L private void finishHim() { try { - phaseFactory.apply(getIterator(results, shardsIts)).run(); + phaseFactory.apply(getIterator(results, shardsIts)).start(); } catch (Exception e) { if (logger.isDebugEnabled()) { logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 8bc058e72c70a..ff9f7bce50383 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -24,12 +24,15 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.transport.TransportRequest; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -202,4 +205,15 @@ public ShardSearchRequest createShardSearchRequest(ShardLevelRequest r) { return shardSearchRequest; } + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new SearchShardTask(id, type, action, getDescription(), parentTaskId, headers); + } + + @Override + public String getDescription() { + // Shard id is enough here, the request itself can be found by looking at the parent task description + return "shardIds[" + shards.stream().map(slr -> slr.shardId).collect(Collectors.toList()) + "]"; + } + } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 893036c3e9c4b..48dd305ec50af 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -130,16 +130,18 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final ActionListener listener) { - if (connection.getVersion().onOrAfter(Version.V_8_0_0)) { + // TODO: use minNodeVersion here to check (i.e. min{connection.getVersion(), targetNode.getVersion()}) + if (connection.getVersion().onOrAfter(Version.V_8_0_0) && + connection.getNode().getVersion().onOrAfter(Version.V_8_0_0)) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); } else { - List shardSearchRequests = request.createShardSearchRequests(); - AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); - AtomicInteger counter = new AtomicInteger(); + // BWC layer: translate into shard-level requests + final List shardSearchRequests = request.createShardSearchRequests(); + final AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); + final AtomicInteger counter = new AtomicInteger(); for (int i = 0; i < shardSearchRequests.size(); i++) { ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); - // TODO: do we need to set parent task etc on this synthetic ShardSearchRequest int finalI = i; sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { @Override diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 0e65b7cd57e50..a4575a6ee228a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -747,8 +747,8 @@ static boolean shouldPreFilterSearchShards(ClusterState clusterState, preFilterShardSize = 1; } return searchRequest.searchType() == QUERY_THEN_FETCH // we can't do this for DFS it needs to fan out to all shards all the time - && (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source)) - && preFilterShardSize < numShards; + /*&& (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source))*/ + && preFilterShardSize <= numShards; } private static boolean hasReadOnlyIndices(String[] indices, ClusterState clusterState) { @@ -809,12 +809,7 @@ private SearchPhase searchAsyncAction( threadPool, clusters); assert action instanceof AbstractSearchAsyncAction; - return new SearchPhase(action.getName()) { - @Override - public void run() { - action.start(); - } - }; + return action; }, clusters, searchService.getCoordinatorRewriteContextProvider(timeProvider::getAbsoluteStartMillis)); } else { final QueryPhaseResultConsumer queryResultConsumer = searchPhaseController.newSearchPhaseResults(executor, diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 1fbf78c8013ec..334d8c28888db 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -1228,17 +1228,17 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { - List responses = new ArrayList<>(); - List failures = new ArrayList<>(); List shardSearchRequests = request.createShardSearchRequests(); + List responses = new ArrayList<>(shardSearchRequests.size()); + List failures = new ArrayList<>(shardSearchRequests.size()); for (int i = 0; i < shardSearchRequests.size(); i++) { ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); CanMatchResponse canMatchResponse; try { canMatchResponse = canMatch(shardSearchRequest); - responses.set(i, canMatchResponse); + responses.add(i, canMatchResponse); } catch (Exception e) { - failures.set(i, e); + failures.add(i, e); } } From 4a8bf1335db5aa09f219a8c96a56bb9b41509a38 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 5 Oct 2021 12:19:58 +0200 Subject: [PATCH 06/28] CCS multi-version test passing --- .../elasticsearch/action/OriginalIndices.java | 16 ++++++++++++++++ .../action/search/CanMatchPhase.java | 2 ++ .../action/search/SearchTransportService.java | 4 ++-- 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/OriginalIndices.java b/server/src/main/java/org/elasticsearch/action/OriginalIndices.java index 7b2f03287bb7a..82627211fd7b4 100644 --- a/server/src/main/java/org/elasticsearch/action/OriginalIndices.java +++ b/server/src/main/java/org/elasticsearch/action/OriginalIndices.java @@ -14,6 +14,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Objects; /** * Used to keep track of original indices within internal (e.g. shard level) requests @@ -67,4 +68,19 @@ public String toString() { ", indicesOptions=" + indicesOptions + '}'; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + OriginalIndices that = (OriginalIndices) o; + return Arrays.equals(indices, that.indices) && Objects.equals(indicesOptions, that.indicesOptions); + } + + @Override + public int hashCode() { + int result = Objects.hash(indicesOptions); + result = 31 * result + Arrays.hashCode(indices); + return result; + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index 35f52bc9bb8be..12e5ef900e89a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -313,6 +313,8 @@ private CanMatchRequest createCanMatchRequest(Map.Entry, L final List shardLevelRequests = entry.getValue().stream().map(this::buildShardLevelRequest) .collect(Collectors.toCollection(ArrayList::new)); + assert entry.getValue().stream().allMatch(ssi -> ssi != null); + assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices() != null); assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); final CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 48dd305ec50af..2dfdf4e1bfcee 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -163,10 +163,10 @@ private void maybeFinish() { for (int i = 0; i < results.length(); i++) { Object o = results.get(i); if (o instanceof SearchService.CanMatchResponse) { - responses.set(i, (SearchService.CanMatchResponse) o); + responses.add(i, (SearchService.CanMatchResponse) o); } else { assert o instanceof Exception; - failures.set(i, (Exception) o); + failures.add(i, (Exception) o); } } CanMatchNodeResponse response = new CanMatchNodeResponse(responses, failures); From a69bcbc489e7ff3e7ee71f91171c884340fb3122 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 12:18:27 +0200 Subject: [PATCH 07/28] unit tests --- .../action/search/CanMatchPhase.java | 154 ++-- .../action/search/CanMatchRequest.java | 62 +- .../action/search/SearchPhase.java | 7 +- .../action/search/TransportSearchAction.java | 2 +- .../action/search/CanMatchPhaseTests.java | 738 ++++++++++++++++++ 5 files changed, 856 insertions(+), 107 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java index 12e5ef900e89a..2e500d3e5c1dd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java @@ -14,9 +14,9 @@ import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.core.Tuple; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.core.Nullable; import org.elasticsearch.index.query.CoordinatorRewriteContext; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.search.SearchService; @@ -40,7 +40,6 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; import java.util.function.Function; @@ -65,6 +64,7 @@ public class CanMatchPhase extends SearchPhase { private final Map aliasFilter; private final SearchTask task; private final Function, SearchPhase> phaseFactory; + private final Executor executor; private final CanMatchSearchPhaseResults results; private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; @@ -75,7 +75,7 @@ public CanMatchPhase(Logger logger, SearchTransportService searchTransportServic Map aliasFilter, Map concreteIndexBoosts, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, - TransportSearchAction.SearchTimeProvider timeProvider, ClusterState clusterState, + TransportSearchAction.SearchTimeProvider timeProvider, SearchTask task, Function, SearchPhase> phaseFactory, SearchResponse.Clusters clusters, CoordinatorRewriteContextProvider coordinatorRewriteContextProvider) { super("can_match"); @@ -92,6 +92,7 @@ public CanMatchPhase(Logger logger, SearchTransportService searchTransportServic this.task = task; this.phaseFactory = phaseFactory; this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; + this.executor = executor; this.shardItIndexMap = new HashMap<>(); results = new CanMatchSearchPhaseResults(shardsIts.size()); @@ -182,18 +183,18 @@ private void checkNoMissingShards() { } } - private Map, List> groupByNode(GroupShardsIterator shards) { - Map, List> requests = new HashMap<>(); + private Map> groupByNode(GroupShardsIterator shards) { + Map> requests = new HashMap<>(); for (int i = 0; i < shards.size(); i++) { final SearchShardIterator shardRoutings = shards.get(i); assert shardRoutings.skip() == false; assert shardItIndexMap.containsKey(shardRoutings); SearchShardTarget target = shardRoutings.nextOrNull(); if (target != null) { - requests.computeIfAbsent(Tuple.tuple(target.getClusterAlias(), target.getNodeId()), + requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), t -> new ArrayList<>()).add(shardRoutings); } else { - requests.computeIfAbsent(Tuple.tuple(null, null), + requests.computeIfAbsent(new SendingTarget(null, null), t -> new ArrayList<>()).add(shardRoutings); } } @@ -202,11 +203,12 @@ private Map, List> groupByNode(GroupS class Round implements Runnable { private final GroupShardsIterator shards; - private final AtomicInteger counter = new AtomicInteger(); + private final CountDown countDown; private final AtomicReferenceArray responses; Round(GroupShardsIterator shards) { this.shards = shards; + this.countDown = new CountDown(shards.size()); this.responses = new AtomicReferenceArray<>(shardsIts.size()); } @@ -224,67 +226,69 @@ public void start() { @Override public void run() { - final Map, List> requests = groupByNode(shards); + final Map> requests = groupByNode(shards); - for (Map.Entry, List> entry : requests.entrySet()) { + for (Map.Entry> entry : requests.entrySet()) { CanMatchRequest canMatchRequest = createCanMatchRequest(entry); - List shardLevelRequests = canMatchRequest.getShardLevelRequests(); + List shardLevelRequests = canMatchRequest.getShardLevelRequests(); - if (entry.getKey().v2() == null) { - // no target node - for (CanMatchRequest.ShardLevelRequest shardLevelRequest : shardLevelRequests) { - results.consumeShardFailure(shardLevelRequest.getShardRequestIndex()); - onOperation(shardLevelRequest.getShardRequestIndex(), null); + if (entry.getKey().nodeId == null) { + // no target node: just mark the requests as failed + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), null); } continue; } - searchTransportService.sendCanMatch(getConnection(entry.getKey().v1(), entry.getKey().v2()), canMatchRequest, - task, new ActionListener<>() { - @Override - public void onResponse(CanMatchNodeResponse canMatchResponse) { - logger.info("Happy response"); - for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); - if (response != null) { - response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); - results.consumeResult(response, () -> { - }); - onOperation(response.getShardIndex(), response); + try { + searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchRequest, + task, new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchResponse) { + for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { + SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); + if (response != null) { + response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + onOperation(response.getShardIndex(), response); + } } - } - for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { - Exception failure = canMatchResponse.getFailures().get(i); - if (failure != null) { - results.consumeShardFailure(i); - onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { + Exception failure = canMatchResponse.getFailures().get(i); + if (failure != null) { + onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + } } } - } - @Override - public void onFailure(Exception e) { - logger.error(e); - for (CanMatchRequest.ShardLevelRequest shardLevelRequest : shardLevelRequests) { - results.consumeShardFailure(shardLevelRequest.getShardRequestIndex()); - onOperationFailed(shardLevelRequest.getShardRequestIndex(), e); + @Override + public void onFailure(Exception e) { + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); + } } } + ); + } catch (Exception e) { + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); } - ); + } } } private void onOperation(int idx, SearchService.CanMatchResponse response) { responses.set(idx, response); - if (counter.incrementAndGet() == responses.length()) { - finishPhase(); - } + results.consumeResult(response, () -> { + if (countDown.countDown()) { + finishPhase(); + } + }); } private void onOperationFailed(int idx, Exception e) { responses.set(idx, e); - if (counter.incrementAndGet() == shards.size()) { + results.consumeShardFailure(idx); + if (countDown.countDown()) { finishPhase(); } } @@ -308,12 +312,37 @@ private void finishPhase() { } } - private CanMatchRequest createCanMatchRequest(Map.Entry, List> entry) { + private static class SendingTarget { + @Nullable + private final String clusterAlias; + @Nullable + private final String nodeId; + + SendingTarget(@Nullable String clusterAlias, @Nullable String nodeId) { + this.clusterAlias = clusterAlias; + this.nodeId = nodeId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SendingTarget that = (SendingTarget) o; + return Objects.equals(clusterAlias, that.clusterAlias) && + Objects.equals(nodeId, that.nodeId); + } + + @Override + public int hashCode() { + return Objects.hash(clusterAlias, nodeId); + } + } + + private CanMatchRequest createCanMatchRequest(Map.Entry> entry) { final SearchShardIterator first = entry.getValue().get(0); - final List shardLevelRequests = - entry.getValue().stream().map(this::buildShardLevelRequest) - .collect(Collectors.toCollection(ArrayList::new)); - assert entry.getValue().stream().allMatch(ssi -> ssi != null); + final List shardLevelRequests = + entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); + assert entry.getValue().stream().allMatch(Objects::nonNull); assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices() != null); assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); @@ -335,11 +364,11 @@ private void finishHim() { private static final float DEFAULT_INDEX_BOOST = 1.0f; - public final CanMatchRequest.ShardLevelRequest buildShardLevelRequest(SearchShardIterator shardIt) { + public final CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - CanMatchRequest.ShardLevelRequest shardRequest = new CanMatchRequest.ShardLevelRequest(shardIt.shardId(), + CanMatchRequest.Shard shardRequest = new CanMatchRequest.Shard(shardIt.shardId(), shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); return shardRequest; } @@ -348,7 +377,7 @@ private boolean checkMinimumVersion(GroupShardsIterator sha for (SearchShardIterator it : shardsIts) { if (it.getTargetNodeIds().isEmpty() == false) { boolean isCompatible = it.getTargetNodeIds().stream().anyMatch(nodeId -> { - Transport.Connection conn = getConnection(it.getClusterAlias(), nodeId); + Transport.Connection conn = getConnection(new SendingTarget(it.getClusterAlias(), nodeId)); return conn == null ? true : conn.getVersion().onOrAfter(request.minCompatibleShardNode()); }); if (isCompatible == false) { @@ -369,8 +398,8 @@ public void start() { request.source().trackTotalHitsUpTo(); // total hits is null in the response if the tracking of total hits is disabled boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; - listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, 0, buildTookInMillis(), - ShardSearchFailure.EMPTY_ARRAY, clusters, null)); + listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, + 0, timeProvider.buildTookInMillis(), ShardSearchFailure.EMPTY_ARRAY, clusters, null)); return; } @@ -385,12 +414,12 @@ public void start() { } - public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + public final void onPhaseFailure(SearchPhase phase, String msg, Exception cause) { listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } - public final Transport.Connection getConnection(String clusterAlias, String nodeId) { - Transport.Connection conn = nodeIdToConnection.apply(clusterAlias, nodeId); + public final Transport.Connection getConnection(SendingTarget sendingTarget) { + Transport.Connection conn = nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId); Version minVersion = request.minCompatibleShardNode(); if (minVersion != null && conn != null && conn.getVersion().before(minVersion)) { throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", minVersion); @@ -402,11 +431,6 @@ private int getNumShards() { return shardsIts.size(); } - long buildTookInMillis() { - return timeProvider.buildTookInMillis(); - } - - private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { private final FixedBitSet possibleMatches; private final MinAndMax[] minAndMaxes; diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index ff9f7bce50383..8645b074c4372 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -8,11 +8,9 @@ package org.elasticsearch.action.search; -import org.elasticsearch.Version; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -30,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -42,7 +39,7 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest private final OriginalIndices originalIndices; private final SearchSourceBuilder source; - private final List shards; + private final List shards; private final SearchType searchType; private final Boolean requestCache; private final boolean allowPartialSearchResults; @@ -52,28 +49,20 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest @Nullable private final String clusterAlias; - public static class ShardLevelRequest implements Writeable { + public static class Shard implements Writeable { private final ShardId shardId; - - public int getShardRequestIndex() { - return shardRequestIndex; - } - private final int shardRequestIndex; - private final float indexBoost; - private final AliasFilter aliasFilter; + private final float indexBoost; private final ShardSearchContextId readerId; private final TimeValue keepAlive; - private final Version channelVersion; - - public ShardLevelRequest( ShardId shardId, - int shardRequestIndex, - AliasFilter aliasFilter, - float indexBoost, - ShardSearchContextId readerId, - TimeValue keepAlive) { + public Shard(ShardId shardId, + int shardRequestIndex, + AliasFilter aliasFilter, + float indexBoost, + ShardSearchContextId readerId, + TimeValue keepAlive) { this.shardId = shardId; this.shardRequestIndex = shardRequestIndex; this.aliasFilter = aliasFilter; @@ -81,48 +70,41 @@ public ShardLevelRequest( ShardId shardId, this.readerId = readerId; this.keepAlive = keepAlive; assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; - this.channelVersion = Version.CURRENT; } - public ShardLevelRequest(StreamInput in) throws IOException { + public Shard(StreamInput in) throws IOException { shardId = new ShardId(in); shardRequestIndex = in.readVInt(); - if (in.getVersion().before(Version.V_8_0_0)) { - // types no longer relevant so ignore - String[] types = in.readStringArray(); - if (types.length > 0) { - throw new IllegalStateException( - "types are no longer supported in search requests but found [" + Arrays.toString(types) + "]"); - } - } aliasFilter = new AliasFilter(in); indexBoost = in.readFloat(); readerId = in.readOptionalWriteable(ShardSearchContextId::new); keepAlive = in.readOptionalTimeValue(); assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; - channelVersion = Version.min(Version.readVersion(in), in.getVersion()); } @Override public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); out.writeVInt(shardRequestIndex); - if (out.getVersion().before(Version.V_8_0_0)) { - // types not supported so send an empty array to previous versions - out.writeStringArray(Strings.EMPTY_ARRAY); - } aliasFilter.writeTo(out); out.writeFloat(indexBoost); out.writeOptionalWriteable(readerId); out.writeOptionalTimeValue(keepAlive); - Version.writeVersion(channelVersion, out); + } + + public int getShardRequestIndex() { + return shardRequestIndex; + } + + public ShardId shardId() { + return shardId; } } public CanMatchRequest( OriginalIndices originalIndices, SearchRequest searchRequest, - List shards, + List shards, int numberOfShards, long nowInMillis, @Nullable String clusterAlias @@ -153,7 +135,7 @@ public CanMatchRequest(StreamInput in) throws IOException { numberOfShards = in.readVInt(); nowInMillis = in.readVLong(); clusterAlias = in.readOptionalString(); - shards = in.readList(ShardLevelRequest::new); + shards = in.readList(Shard::new); } @Override @@ -187,7 +169,7 @@ public IndicesOptions indicesOptions() { return originalIndices.indicesOptions(); } - public List getShardLevelRequests() { + public List getShardLevelRequests() { return shards; } @@ -195,7 +177,7 @@ public List createShardSearchRequests() { return shards.stream().map(this::createShardSearchRequest).collect(Collectors.toList()); } - public ShardSearchRequest createShardSearchRequest(ShardLevelRequest r) { + public ShardSearchRequest createShardSearchRequest(Shard r) { ShardSearchRequest shardSearchRequest = new ShardSearchRequest( originalIndices, r.shardId, r.shardRequestIndex, numberOfShards, searchType, source, requestCache, r.aliasFilter, r.indexBoost, allowPartialSearchResults, scroll, diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 3de90d727fdfb..88da2fdfa3a9e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -10,6 +10,7 @@ import org.elasticsearch.core.CheckedRunnable; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Objects; /** @@ -30,6 +31,10 @@ public String getName() { } public void start() { - + try { + run(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a4575a6ee228a..a7c60261ce3dd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -793,7 +793,7 @@ private SearchPhase searchAsyncAction( if (preFilter) { return new CanMatchPhase(logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators, - timeProvider, clusterState, task, (iter) -> { + timeProvider, task, (iter) -> { SearchPhase action = searchAsyncAction( task, searchRequest, diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java new file mode 100644 index 0000000000000..a23fba7ead913 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java @@ -0,0 +1,738 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.search; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.shard.IndexLongFieldRange; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardLongFieldRange; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.sort.MinAndMax; +import org.elasticsearch.search.sort.SortBuilders; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.Transport; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.elasticsearch.action.search.SearchAsyncActionTests.getShardsIter; +import static org.elasticsearch.core.Types.forciblyCast; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.mockito.Mockito.mock; + +public class CanMatchPhaseTests extends ESTestCase { + + private final CoordinatorRewriteContextProvider EMPTY_CONTEXT_PROVIDER = new StaticCoordinatorRewriteContextProviderBuilder().build(); + + public void testFilterShards() throws InterruptedException { + + final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), + System::nanoTime); + + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = randomBoolean() ? null : + new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + final boolean shard1 = randomBoolean(); + final boolean shard2 = randomBoolean(); + + final AtomicInteger numRequests = new AtomicInteger(); + SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + @Override + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + numRequests.incrementAndGet(); + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + responses.add(new SearchService.CanMatchResponse(shard.shardId().id() == 0 ? shard1 : + shard2, null)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), + 2, randomBoolean(), primaryNode, replicaNode); + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(true); + + CanMatchPhase canMatchPhase = new CanMatchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + searchRequest, null, shardsIter, timeProvider,null, + (iter) -> new SearchPhase("test") { + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); + + canMatchPhase.start(); + latch.await(); + + assertThat(numRequests.get(), replicaNode == null ? equalTo(1) : lessThanOrEqualTo(2)); + + if (shard1 && shard2) { + for (SearchShardIterator i : result.get()) { + assertFalse(i.skip()); + } + } else if (shard1 == false && shard2 == false) { + assertFalse(result.get().get(0).skip()); + assertTrue(result.get().get(1).skip()); + } else { + assertEquals(0, result.get().get(0).shardId().id()); + assertEquals(1, result.get().get(1).shardId().id()); + assertEquals(shard1, result.get().get(0).skip() == false); + assertEquals(shard2, result.get().get(1).skip() == false); + } + } + + public void testFilterWithFailure() throws InterruptedException { + final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), + System::nanoTime); + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + final boolean shard1 = randomBoolean(); + final boolean useReplicas = randomBoolean(); + final boolean fullFailure = randomBoolean(); + SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + @Override + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + if (fullFailure && randomBoolean()) { + throw new IllegalArgumentException("boom"); + } + final SearchService.CanMatchResponse[] responses = + new SearchService.CanMatchResponse[request.getShardLevelRequests().size()]; + final Exception[] failures = new Exception[request.getShardLevelRequests().size()]; + for (int i = 0; i < request.getShardLevelRequests().size(); i++) { + CanMatchRequest.Shard shard = request.getShardLevelRequests().get(i); + boolean throwException = shard.shardId().id() != 0; + if (throwException) { + failures[i] = new NullPointerException(); + } else { + responses[i] = new SearchService.CanMatchResponse(shard1, null); + } + } + + new Thread(() -> { + if (fullFailure) { + listener.onFailure(new NullPointerException()); + } else { + listener.onResponse(new CanMatchNodeResponse(Arrays.asList(responses), Arrays.asList(failures))); + } + }).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = getShardsIter("idx", + new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), + 2, useReplicas, primaryNode, replicaNode); + + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(true); + + CanMatchPhase canMatchPhase = new CanMatchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + searchRequest, null, shardsIter, timeProvider,null, + (iter) -> new SearchPhase("test") { + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); + + canMatchPhase.start(); + latch.await(); + + assertEquals(0, result.get().get(0).shardId().id()); + assertEquals(1, result.get().get(1).shardId().id()); + if (fullFailure) { + assertFalse(result.get().get(0).skip()); // never skip the failure + } else { + assertEquals(shard1, result.get().get(0).skip() == false); + } + assertFalse(result.get().get(1).skip()); // never skip the failure + } + + public void testSortShards() throws InterruptedException { + final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), + System::nanoTime); + + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + + for (SortOrder order : SortOrder.values()) { + List shardIds = new ArrayList<>(); + List> minAndMaxes = new ArrayList<>(); + Set shardToSkip = new HashSet<>(); + + SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + @Override + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + Long min = rarely() ? null : randomLong(); + Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); + MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); + boolean canMatch = frequently(); + synchronized (shardIds) { + shardIds.add(shard.shardId()); + minAndMaxes.add(minMax); + if (canMatch == false) { + shardToSkip.add(shard.shardId()); + } + } + + responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = getShardsIter("logs", + new OriginalIndices(new String[]{"logs"}, SearchRequest.DEFAULT_INDICES_OPTIONS), + randomIntBetween(2, 20), randomBoolean(), primaryNode, replicaNode); + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); + searchRequest.allowPartialSearchResults(true); + + CanMatchPhase canMatchPhase = new CanMatchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + searchRequest, null, shardsIter, timeProvider, null, + (iter) -> new SearchPhase("test") { + @Override + public void run() { + result.set(iter); + latch.countDown(); + } + }, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); + + canMatchPhase.start(); + latch.await(); + ShardId[] expected = IntStream.range(0, shardIds.size()) + .boxed() + .sorted(Comparator.comparing(minAndMaxes::get, forciblyCast(MinAndMax.getComparator(order))).thenComparing(shardIds::get)) + .map(shardIds::get) + .toArray(ShardId[]::new); + if (shardToSkip.size() == expected.length) { + // we need at least one shard to produce the empty result for aggs + shardToSkip.remove(new ShardId("logs", "_na_", 0)); + } + int pos = 0; + for (SearchShardIterator i : result.get()) { + assertEquals(shardToSkip.contains(i.shardId()), i.skip()); + assertEquals(expected[pos++], i.shardId()); + } + } + } + + public void testInvalidSortShards() throws InterruptedException { + final TransportSearchAction.SearchTimeProvider timeProvider = + new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); + + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + + for (SortOrder order : SortOrder.values()) { + int numShards = randomIntBetween(2, 20); + List shardIds = new ArrayList<>(); + Set shardToSkip = new HashSet<>(); + + SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + @Override + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + final MinAndMax minMax; + if (shard.shardId().id() == numShards-1) { + minMax = new MinAndMax<>(new BytesRef("bar"), new BytesRef("baz")); + } else { + Long min = randomLong(); + Long max = randomLongBetween(min, Long.MAX_VALUE); + minMax = new MinAndMax<>(min, max); + } + boolean canMatch = frequently(); + synchronized (shardIds) { + shardIds.add(shard.shardId()); + if (canMatch == false) { + shardToSkip.add(shard.shardId()); + } + } + responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + } + }; + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + GroupShardsIterator shardsIter = getShardsIter("logs", + new OriginalIndices(new String[]{"logs"}, SearchRequest.DEFAULT_INDICES_OPTIONS), + numShards, randomBoolean(), primaryNode, replicaNode); + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); + searchRequest.allowPartialSearchResults(true); + + CanMatchPhase canMatchPhase = new CanMatchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), + Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + searchRequest, null, shardsIter, timeProvider, null, + (iter) -> new SearchPhase("test") { + @Override + public void run() { + result.set(iter); + latch.countDown(); + } + }, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); + + canMatchPhase.start(); + latch.await(); + int shardId = 0; + for (SearchShardIterator i : result.get()) { + assertThat(i.shardId().id(), equalTo(shardId++)); + assertEquals(shardToSkip.contains(i.shardId()), i.skip()); + } + assertThat(result.get().size(), equalTo(numShards)); + } + } + + public void testCanMatchFilteringOnCoordinatorThatCanBeSkipped() throws Exception { + Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); + Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); + DataStream dataStream = + new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); + + List regularIndices = + randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); + + long indexMinTimestamp = randomLongBetween(0, 5000); + long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2); + StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); + String timestampFieldName = dataStream.getTimeStampField().getName(); + for (Index dataStreamIndex : dataStream.getIndices()) { + contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); + } + + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); + // We query a range outside of the timestamp range covered by both datastream indices + rangeQueryBuilder + .from(indexMaxTimestamp + 1) + .to(indexMaxTimestamp + 2); + + BoolQueryBuilder queryBuilder = new BoolQueryBuilder() + .filter(rangeQueryBuilder); + + if (randomBoolean()) { + // Add an additional filter that cannot be evaluated in the coordinator but shouldn't + // affect the end result as we're filtering + queryBuilder.filter(new TermQueryBuilder("fake", "value")); + } + + assignShardsAndExecuteCanMatchPhase(dataStream, regularIndices, contextProviderBuilder.build(), queryBuilder, + (updatedSearchShardIterators, requests) -> { + List skippedShards = updatedSearchShardIterators.stream() + .filter(SearchShardIterator::skip) + .collect(Collectors.toList());; + + List nonSkippedShards = updatedSearchShardIterators.stream() + .filter(searchShardIterator -> searchShardIterator.skip() == false) + .collect(Collectors.toList());; + + int regularIndexShardCount = (int) updatedSearchShardIterators.stream() + .filter(s -> regularIndices.contains(s.shardId().getIndex())) + .count(); + + // When all the shards can be skipped we should query at least 1 + // in order to get a valid search response. + if (regularIndexShardCount == 0) { + assertThat(nonSkippedShards.size(), equalTo(1)); + } else { + boolean allNonSkippedShardsAreFromRegularIndices = nonSkippedShards.stream() + .allMatch(shardIterator -> regularIndices.contains(shardIterator.shardId().getIndex())); + + assertThat(allNonSkippedShardsAreFromRegularIndices, equalTo(true)); + } + + boolean allSkippedShardAreFromDataStream = skippedShards.stream() + .allMatch(shardIterator -> dataStream.getIndices().contains(shardIterator.shardId().getIndex())); + assertThat(allSkippedShardAreFromDataStream, equalTo(true)); + + boolean allRequestsWereTriggeredAgainstRegularIndices = requests.stream() + .allMatch(request -> regularIndices.contains(request.shardId().getIndex())); + assertThat(allRequestsWereTriggeredAgainstRegularIndices, equalTo(true)); + }); + } + + public void testCanMatchFilteringOnCoordinatorParsingFails() throws Exception { + Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); + Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); + DataStream dataStream = + new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); + + List regularIndices = + randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); + + long indexMinTimestamp = randomLongBetween(0, 5000); + long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2); + StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); + String timestampFieldName = dataStream.getTimeStampField().getName(); + for (Index dataStreamIndex : dataStream.getIndices()) { + contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); + } + + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); + // Query with a non default date format + rangeQueryBuilder + .from("2020-1-01") + .to("2021-1-01"); + + BoolQueryBuilder queryBuilder = new BoolQueryBuilder() + .filter(rangeQueryBuilder); + + if (randomBoolean()) { + // Add an additional filter that cannot be evaluated in the coordinator but shouldn't + // affect the end result as we're filtering + queryBuilder.filter(new TermQueryBuilder("fake", "value")); + } + + assignShardsAndExecuteCanMatchPhase(dataStream, + regularIndices, + contextProviderBuilder.build(), + queryBuilder, + this::assertAllShardsAreQueried + ); + } + + public void testCanMatchFilteringOnCoordinatorThatCanNotBeSkipped() throws Exception { + // Generate indices + Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); + Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); + DataStream dataStream = + new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); + + List regularIndices = + randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); + + long indexMinTimestamp = 10; + long indexMaxTimestamp = 20; + StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); + String timestampFieldName = dataStream.getTimeStampField().getName(); + for (Index dataStreamIndex : dataStream.getIndices()) { + contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); + } + + BoolQueryBuilder queryBuilder = new BoolQueryBuilder(); + // Query inside of the data stream index range + if (randomBoolean()) { + // Query generation + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); + // We query a range within the timestamp range covered by both datastream indices + rangeQueryBuilder + .from(indexMinTimestamp) + .to(indexMaxTimestamp); + + queryBuilder.filter(rangeQueryBuilder); + + if (randomBoolean()) { + // Add an additional filter that cannot be evaluated in the coordinator but shouldn't + // affect the end result as we're filtering + queryBuilder.filter(new TermQueryBuilder("fake", "value")); + } + } else { + // We query a range outside of the timestamp range covered by both datastream indices + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName) + .from(indexMaxTimestamp + 1) + .to(indexMaxTimestamp + 2); + + TermQueryBuilder termQueryBuilder = new TermQueryBuilder("fake", "value"); + + // This is always evaluated as true in the coordinator as we cannot determine there if + // the term query clause is false. + queryBuilder.should(rangeQueryBuilder) + .should(termQueryBuilder); + } + + assignShardsAndExecuteCanMatchPhase(dataStream, + regularIndices, + contextProviderBuilder.build(), + queryBuilder, + this::assertAllShardsAreQueried + ); + } + + private void assertAllShardsAreQueried(List updatedSearchShardIterators, List requests) { + int skippedShards = (int) updatedSearchShardIterators.stream() + .filter(SearchShardIterator::skip) + .count(); + + assertThat(skippedShards, equalTo(0)); + + int nonSkippedShards = (int) updatedSearchShardIterators.stream() + .filter(searchShardIterator -> searchShardIterator.skip() == false) + .count(); + + assertThat(nonSkippedShards, equalTo(updatedSearchShardIterators.size())); + + int shardsWithPrimariesAssigned = (int) updatedSearchShardIterators.stream() + .filter(s -> s.size() > 0) + .count(); + assertThat(requests.size(), equalTo(shardsWithPrimariesAssigned)); + } + + private > + void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, + List regularIndices, + CoordinatorRewriteContextProvider contextProvider, + AbstractQueryBuilder query, + BiConsumer, + List> canMatchResultsConsumer) throws Exception { + Map lookup = new ConcurrentHashMap<>(); + DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); + lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); + + List indicesToSearch = new ArrayList<>(); + indicesToSearch.add(dataStream.getName()); + for (Index regularIndex : regularIndices) { + indicesToSearch.add(regularIndex.getName()); + } + + String[] indices = indicesToSearch.toArray(new String[0]); + OriginalIndices originalIndices = new OriginalIndices(indices, SearchRequest.DEFAULT_INDICES_OPTIONS); + + boolean atLeastOnePrimaryAssigned = false; + final List originalShardIters = new ArrayList<>(); + for (Index dataStreamIndex : dataStream.getIndices()) { + // If we have to execute the can match request against all the shards + // and none is assigned, the phase is considered as failed meaning that the next phase won't be executed + boolean withAssignedPrimaries = randomBoolean() || atLeastOnePrimaryAssigned == false; + int numShards = randomIntBetween(1, 6); + originalShardIters.addAll( + getShardsIter(dataStreamIndex, + originalIndices, + numShards, + false, + withAssignedPrimaries ? primaryNode : null, + null) + ); + atLeastOnePrimaryAssigned |= withAssignedPrimaries; + } + + for (Index regularIndex : regularIndices) { + originalShardIters.addAll( + getShardsIter(regularIndex, + originalIndices, + randomIntBetween(1, 6), + randomBoolean(), + primaryNode, + replicaNode) + ); + } + GroupShardsIterator shardsIter = GroupShardsIterator.sortAndCreate(originalShardIters); + + final SearchRequest searchRequest = new SearchRequest(); + searchRequest.indices(indices); + searchRequest.allowPartialSearchResults(true); + + final AliasFilter aliasFilter; + if (randomBoolean()) { + // Apply the query on the request body + SearchSourceBuilder searchSourceBuilder = SearchSourceBuilder.searchSource(); + searchSourceBuilder.query(query); + searchRequest.source(searchSourceBuilder); + + // Sometimes apply the same query in the alias filter too + aliasFilter = new AliasFilter(randomBoolean() ? query : null, Strings.EMPTY_ARRAY); + } else { + // Apply the query as an alias filter + aliasFilter = new AliasFilter(query, Strings.EMPTY_ARRAY); + } + + Map aliasFilters = new HashMap<>(); + for (Index dataStreamIndex : dataStream.getIndices()) { + aliasFilters.put(dataStreamIndex.getUUID(), aliasFilter); + } + + for (Index regularIndex : regularIndices) { + aliasFilters.put(regularIndex.getUUID(), aliasFilter); + } + + // We respond by default that the query can match + final List requests = Collections.synchronizedList(new ArrayList<>()); + SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + @Override + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + requests.add(request.createShardSearchRequest(shard)); + responses.add(new SearchService.CanMatchResponse(true, null)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + } + }; + + final TransportSearchAction.SearchTimeProvider timeProvider = + new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); + + AtomicReference> result = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + CanMatchPhase canMatchPhase = new CanMatchPhase(logger, + searchTransportService, + (clusterAlias, node) -> lookup.get(node), + aliasFilters, + Collections.emptyMap(), + EsExecutors.DIRECT_EXECUTOR_SERVICE, + searchRequest, + null, + shardsIter, + timeProvider, + null, + (iter) -> new SearchPhase("test") { + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + } + }, + SearchResponse.Clusters.EMPTY, + contextProvider); + + canMatchPhase.start(); + latch.await(); + + List updatedSearchShardIterators = new ArrayList<>(); + for (SearchShardIterator updatedSearchShardIterator : result.get()) { + updatedSearchShardIterators.add(updatedSearchShardIterator); + } + + canMatchResultsConsumer.accept(updatedSearchShardIterators, requests); + } + + private static class StaticCoordinatorRewriteContextProviderBuilder { + private ClusterState clusterState = ClusterState.EMPTY_STATE; + private final Map fields = new HashMap<>(); + + private void addIndexMinMaxTimestamps(Index index, String fieldName, long minTimeStamp, long maxTimestamp) { + if (clusterState.metadata().index(index) != null) { + throw new IllegalArgumentException("Min/Max timestamps for " + index + " were already defined"); + } + + IndexLongFieldRange timestampRange = IndexLongFieldRange.NO_SHARDS + .extendWithShardRange(0, 1, ShardLongFieldRange.of(minTimeStamp, maxTimestamp)); + + Settings.Builder indexSettings = settings(Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()); + + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(index.getName()) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(0) + .timestampRange(timestampRange); + + Metadata.Builder metadataBuilder = + Metadata.builder(clusterState.metadata()) + .put(indexMetadataBuilder); + + clusterState = ClusterState.builder(clusterState) + .metadata(metadataBuilder) + .build(); + + fields.put(index, new DateFieldMapper.DateFieldType(fieldName)); + } + + public CoordinatorRewriteContextProvider build() { + return new CoordinatorRewriteContextProvider(NamedXContentRegistry.EMPTY, + mock(NamedWriteableRegistry.class), + mock(Client.class), + System::currentTimeMillis, + () -> clusterState, + fields::get); + } + } +} From fe56d5a2473a776ce62dc820adef6b9af27bb1cc Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 12:20:09 +0200 Subject: [PATCH 08/28] simpler diff? --- .../action/search/CanMatchPhase.java | 559 ------------- .../search/CanMatchPreFilterSearchPhase.java | 552 ++++++++++--- .../action/search/TransportSearchAction.java | 2 +- .../action/search/CanMatchPhaseTests.java | 738 ------------------ .../CanMatchPreFilterSearchPhaseTests.java | 272 +++---- 5 files changed, 539 insertions(+), 1584 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java delete mode 100644 server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java deleted file mode 100644 index 2e500d3e5c1dd..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPhase.java +++ /dev/null @@ -1,559 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.util.CollectionUtil; -import org.apache.lucene.util.FixedBitSet; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.index.query.CoordinatorRewriteContext; -import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; -import org.elasticsearch.search.SearchService; -import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.internal.AliasFilter; -import org.elasticsearch.search.internal.InternalSearchResponse; -import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.search.sort.FieldSortBuilder; -import org.elasticsearch.search.sort.MinAndMax; -import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.transport.Transport; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -import static org.elasticsearch.core.Types.forciblyCast; - -public class CanMatchPhase extends SearchPhase { - - private final Logger logger; - private final SearchRequest request; - private final GroupShardsIterator shardsIts; - private final ActionListener listener; - private final SearchResponse.Clusters clusters; - private final TransportSearchAction.SearchTimeProvider timeProvider; - private final BiFunction nodeIdToConnection; - private final SearchTransportService searchTransportService; - private final Map shardItIndexMap; - private final Map concreteIndexBoosts; - private final Map aliasFilter; - private final SearchTask task; - private final Function, SearchPhase> phaseFactory; - private final Executor executor; - - private final CanMatchSearchPhaseResults results; - private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; - - - public CanMatchPhase(Logger logger, SearchTransportService searchTransportService, - BiFunction nodeIdToConnection, - Map aliasFilter, Map concreteIndexBoosts, - Executor executor, SearchRequest request, - ActionListener listener, GroupShardsIterator shardsIts, - TransportSearchAction.SearchTimeProvider timeProvider, - SearchTask task, Function, SearchPhase> phaseFactory, - SearchResponse.Clusters clusters, CoordinatorRewriteContextProvider coordinatorRewriteContextProvider) { - super("can_match"); - this.logger = logger; - this.searchTransportService = searchTransportService; - this.nodeIdToConnection = nodeIdToConnection; - this.request = request; - this.listener = listener; - this.shardsIts = shardsIts; - this.clusters = clusters; - this.timeProvider = timeProvider; - this.concreteIndexBoosts = concreteIndexBoosts; - this.aliasFilter = aliasFilter; - this.task = task; - this.phaseFactory = phaseFactory; - this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; - this.executor = executor; - this.shardItIndexMap = new HashMap<>(); - results = new CanMatchSearchPhaseResults(shardsIts.size()); - - // we compute the shard index based on the natural order of the shards - // that participate in the search request. This means that this number is - // consistent between two requests that target the same shards. - List naturalOrder = new ArrayList<>(); - shardsIts.iterator().forEachRemaining(naturalOrder::add); - CollectionUtil.timSort(naturalOrder); - for (int i = 0; i < naturalOrder.size(); i++) { - shardItIndexMap.put(naturalOrder.get(i), i); - } - } - - @Override - public void run() throws IOException { - if (shardsIts.size() > 0) { - checkNoMissingShards(); - Version version = request.minCompatibleShardNode(); - if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { - if (checkMinimumVersion(shardsIts) == false) { - throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", - request.minCompatibleShardNode()); - } - } - - runCoordinationPhase(); - } - } - - private void runCoordinationPhase() { - final List matchedShardLevelRequests = new ArrayList<>(); - for (SearchShardIterator searchShardIterator : shardsIts) { - final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, - Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), - timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); - List shardSearchRequests = canMatchRequest.createShardSearchRequests(); - for (int i = 0; i < shardSearchRequests.size(); i++) { - ShardSearchRequest request = shardSearchRequests.get(i); - - boolean canMatch = true; - CoordinatorRewriteContext coordinatorRewriteContext = - coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); - if (coordinatorRewriteContext != null) { - try { - canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); - } catch (Exception e) { - // treat as if shard is still a potential match - } - } - if (canMatch) { - matchedShardLevelRequests.add(searchShardIterator); - } else { - SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); - result.setShardIndex(request.shardRequestIndex()); - results.consumeResult(result, () -> {}); - } - } - } - - if (matchedShardLevelRequests.isEmpty() == false) { - new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).start(); - } else { - finishHim(); - } - } - - private void checkNoMissingShards() { - assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (request.allowPartialSearchResults() == false) { - final StringBuilder missingShards = new StringBuilder(); - // Fail-fast verification of all shards being available - for (int index = 0; index < shardsIts.size(); index++) { - final SearchShardIterator shardRoutings = shardsIts.get(index); - if (shardRoutings.size() == 0) { - if (missingShards.length() > 0) { - missingShards.append(", "); - } - missingShards.append(shardRoutings.shardId()); - } - } - if (missingShards.length() > 0) { - //Status red - shard is missing all copies and would produce partial results for an index search - final String msg = "Search rejected due to missing shards ["+ missingShards + - "]. Consider using `allow_partial_search_results` setting to bypass this error."; - throw new SearchPhaseExecutionException(getName(), msg, null, ShardSearchFailure.EMPTY_ARRAY); - } - } - } - - private Map> groupByNode(GroupShardsIterator shards) { - Map> requests = new HashMap<>(); - for (int i = 0; i < shards.size(); i++) { - final SearchShardIterator shardRoutings = shards.get(i); - assert shardRoutings.skip() == false; - assert shardItIndexMap.containsKey(shardRoutings); - SearchShardTarget target = shardRoutings.nextOrNull(); - if (target != null) { - requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), - t -> new ArrayList<>()).add(shardRoutings); - } else { - requests.computeIfAbsent(new SendingTarget(null, null), - t -> new ArrayList<>()).add(shardRoutings); - } - } - return requests; - } - - class Round implements Runnable { - private final GroupShardsIterator shards; - private final CountDown countDown; - private final AtomicReferenceArray responses; - - Round(GroupShardsIterator shards) { - this.shards = shards; - this.countDown = new CountDown(shards.size()); - this.responses = new AtomicReferenceArray<>(shardsIts.size()); - } - - public void start() { - try { - run(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); - } - onPhaseFailure(CanMatchPhase.this, "", e); - } - } - - - @Override - public void run() { - final Map> requests = groupByNode(shards); - - for (Map.Entry> entry : requests.entrySet()) { - CanMatchRequest canMatchRequest = createCanMatchRequest(entry); - List shardLevelRequests = canMatchRequest.getShardLevelRequests(); - - if (entry.getKey().nodeId == null) { - // no target node: just mark the requests as failed - for (CanMatchRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), null); - } - continue; - } - - try { - searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchRequest, - task, new ActionListener<>() { - @Override - public void onResponse(CanMatchNodeResponse canMatchResponse) { - for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); - if (response != null) { - response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); - onOperation(response.getShardIndex(), response); - } - } - for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { - Exception failure = canMatchResponse.getFailures().get(i); - if (failure != null) { - onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); - } - } - } - - @Override - public void onFailure(Exception e) { - for (CanMatchRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), e); - } - } - } - ); - } catch (Exception e) { - for (CanMatchRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), e); - } - } - } - } - - private void onOperation(int idx, SearchService.CanMatchResponse response) { - responses.set(idx, response); - results.consumeResult(response, () -> { - if (countDown.countDown()) { - finishPhase(); - } - }); - } - - private void onOperationFailed(int idx, Exception e) { - responses.set(idx, e); - results.consumeShardFailure(idx); - if (countDown.countDown()) { - finishPhase(); - } - } - - private void finishPhase() { - List remainingShards = new ArrayList<>(); - for (SearchShardIterator ssi : shards) { - int shardIndex = shardItIndexMap.get(ssi); - Object resp = responses.get(shardIndex); - if (resp instanceof Exception) { - // do something meaningful - remainingShards.add(ssi); - } - } - if (remainingShards.isEmpty()) { - finishHim(); - } else { - // trigger another round - new Round(new GroupShardsIterator<>(remainingShards)).start(); - } - } - } - - private static class SendingTarget { - @Nullable - private final String clusterAlias; - @Nullable - private final String nodeId; - - SendingTarget(@Nullable String clusterAlias, @Nullable String nodeId) { - this.clusterAlias = clusterAlias; - this.nodeId = nodeId; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SendingTarget that = (SendingTarget) o; - return Objects.equals(clusterAlias, that.clusterAlias) && - Objects.equals(nodeId, that.nodeId); - } - - @Override - public int hashCode() { - return Objects.hash(clusterAlias, nodeId); - } - } - - private CanMatchRequest createCanMatchRequest(Map.Entry> entry) { - final SearchShardIterator first = entry.getValue().get(0); - final List shardLevelRequests = - entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); - assert entry.getValue().stream().allMatch(Objects::nonNull); - assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices() != null); - assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); - assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); - final CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, - shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); - return canMatchRequest; - } - - private void finishHim() { - try { - phaseFactory.apply(getIterator(results, shardsIts)).start(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); - } - onPhaseFailure(this, "", e); - } - } - - private static final float DEFAULT_INDEX_BOOST = 1.0f; - - public final CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { - AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); - assert filter != null; - float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - CanMatchRequest.Shard shardRequest = new CanMatchRequest.Shard(shardIt.shardId(), - shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); - return shardRequest; - } - - private boolean checkMinimumVersion(GroupShardsIterator shardsIts) { - for (SearchShardIterator it : shardsIts) { - if (it.getTargetNodeIds().isEmpty() == false) { - boolean isCompatible = it.getTargetNodeIds().stream().anyMatch(nodeId -> { - Transport.Connection conn = getConnection(new SendingTarget(it.getClusterAlias(), nodeId)); - return conn == null ? true : conn.getVersion().onOrAfter(request.minCompatibleShardNode()); - }); - if (isCompatible == false) { - return false; - } - } - } - return true; - } - - @Override - public void start() { - if (getNumShards() == 0) { - //no search shards to search on, bail with empty response - //(it happens with search across _all with no indices around and consistent with broadcast operations) - int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : - request.source().trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : - request.source().trackTotalHitsUpTo(); - // total hits is null in the response if the tracking of total hits is disabled - boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; - listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, - 0, timeProvider.buildTookInMillis(), ShardSearchFailure.EMPTY_ARRAY, clusters, null)); - return; - } - - try { - run(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); - } - onPhaseFailure(this, "", e); - } - } - - - public final void onPhaseFailure(SearchPhase phase, String msg, Exception cause) { - listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); - } - - public final Transport.Connection getConnection(SendingTarget sendingTarget) { - Transport.Connection conn = nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId); - Version minVersion = request.minCompatibleShardNode(); - if (minVersion != null && conn != null && conn.getVersion().before(minVersion)) { - throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", minVersion); - } - return conn; - } - - private int getNumShards() { - return shardsIts.size(); - } - - private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { - private final FixedBitSet possibleMatches; - private final MinAndMax[] minAndMaxes; - private int numPossibleMatches; - - CanMatchSearchPhaseResults(int size) { - super(size); - possibleMatches = new FixedBitSet(size); - minAndMaxes = new MinAndMax[size]; - } - - @Override - void consumeResult(SearchService.CanMatchResponse result, Runnable next) { - try { - consumeResult(result.getShardIndex(), result.canMatch(), result.estimatedMinAndMax()); - } finally { - next.run(); - } - } - - @Override - boolean hasResult(int shardIndex) { - return false; // unneeded - } - - @Override - void consumeShardFailure(int shardIndex) { - // we have to carry over shard failures in order to account for them in the response. - consumeResult(shardIndex, true, null); - } - - synchronized void consumeResult(int shardIndex, boolean canMatch, MinAndMax minAndMax) { - if (canMatch) { - possibleMatches.set(shardIndex); - numPossibleMatches++; - } - minAndMaxes[shardIndex] = minAndMax; - } - - synchronized int getNumPossibleMatches() { - return numPossibleMatches; - } - - synchronized FixedBitSet getPossibleMatches() { - return possibleMatches; - } - - @Override - Stream getSuccessfulResults() { - return Stream.empty(); - } - } - - private GroupShardsIterator getIterator(CanMatchSearchPhaseResults results, - GroupShardsIterator shardsIts) { - int cardinality = results.getNumPossibleMatches(); - FixedBitSet possibleMatches = results.getPossibleMatches(); - if (cardinality == 0) { - // this is a special case where we have no hit but we need to get at least one search response in order - // to produce a valid search result with all the aggs etc. - // Since it's possible that some of the shards that we're skipping are - // unavailable, we would try to query the node that at least has some - // shards available in order to produce a valid search result. - int shardIndexToQuery = 0; - for (int i = 0; i < shardsIts.size(); i++) { - if (shardsIts.get(i).size() > 0) { - shardIndexToQuery = i; - break; - } - } - possibleMatches.set(shardIndexToQuery); - } - SearchSourceBuilder source = request.source(); - int i = 0; - for (SearchShardIterator iter : shardsIts) { - if (possibleMatches.get(i++)) { - iter.reset(); - } else { - iter.resetAndSkip(); - } - } - if (shouldSortShards(results.minAndMaxes) == false) { - return shardsIts; - } - FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); - return new GroupShardsIterator<>(sortShards(shardsIts, results.minAndMaxes, fieldSort.order())); - } - - private static List sortShards(GroupShardsIterator shardsIts, - MinAndMax[] minAndMaxes, - SortOrder order) { - return IntStream.range(0, shardsIts.size()) - .boxed() - .sorted(shardComparator(shardsIts, minAndMaxes, order)) - .map(shardsIts::get) - .collect(Collectors.toList()); - } - - private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { - Class clazz = null; - for (MinAndMax minAndMax : minAndMaxes) { - if (clazz == null) { - clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); - } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { - // we don't support sort values that mix different types (e.g.: long/double, numeric/keyword). - // TODO: we could fail the request because there is a high probability - // that the merging of topdocs will fail later for the same reason ? - return false; - } - } - return clazz != null; - } - - private static Comparator shardComparator(GroupShardsIterator shardsIts, - MinAndMax[] minAndMaxes, - SortOrder order) { - final Comparator comparator = Comparator.comparing( - index -> minAndMaxes[index], - forciblyCast(MinAndMax.getComparator(order)) - ); - - return comparator.thenComparing(index -> shardsIts.get(index)); - } - -} - diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 09ae052bd1d5e..0d66356ba6226 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -5,31 +5,42 @@ * in compliance with, at your election, the Elastic License 2.0 or the Server * Side Public License, v 1. */ + package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.core.Releasable; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.core.Nullable; import org.elasticsearch.index.query.CoordinatorRewriteContext; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.search.SearchService; -import org.elasticsearch.search.SearchService.CanMatchResponse; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.InternalSearchResponse; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.MinAndMax; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.transport.Transport; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -38,164 +49,390 @@ import static org.elasticsearch.core.Types.forciblyCast; -/** - * This search phase can be used as an initial search phase to pre-filter search shards based on query rewriting. - * The queries are rewritten against the shards and based on the rewrite result shards might be able to be excluded - * from the search. The extra round trip to the search shards is very cheap and is not subject to rejections - * which allows to fan out to more shards at the same time without running into rejections even if we are hitting a - * large portion of the clusters indices. - * This phase can also be used to pre-sort shards based on min/max values in each shard of the provided primary sort. - * When the query primary sort is perform on a field, this phase extracts the min/max value in each shard and - * sort them according to the provided order. This can be useful for instance to ensure that shards that contain recent - * data are executed first when sorting by descending timestamp. - */ -final class CanMatchPreFilterSearchPhase extends AbstractSearchAsyncAction { +public class CanMatchPreFilterSearchPhase extends SearchPhase { - private final Function, SearchPhase> phaseFactory; + private final Logger logger; + private final SearchRequest request; private final GroupShardsIterator shardsIts; + private final ActionListener listener; + private final SearchResponse.Clusters clusters; + private final TransportSearchAction.SearchTimeProvider timeProvider; + private final BiFunction nodeIdToConnection; + private final SearchTransportService searchTransportService; + private final Map shardItIndexMap; + private final Map concreteIndexBoosts; + private final Map aliasFilter; + private final SearchTask task; + private final Function, SearchPhase> phaseFactory; + private final Executor executor; + + private final CanMatchSearchPhaseResults results; private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; - CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService, - BiFunction nodeIdToConnection, - Map aliasFilter, Map concreteIndexBoosts, - Executor executor, SearchRequest request, - ActionListener listener, GroupShardsIterator shardsIts, - TransportSearchAction.SearchTimeProvider timeProvider, ClusterState clusterState, - SearchTask task, Function, SearchPhase> phaseFactory, - SearchResponse.Clusters clusters, CoordinatorRewriteContextProvider coordinatorRewriteContextProvider) { - //We set max concurrent shard requests to the number of shards so no throttling happens for can_match requests - super("can_match", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, - executor, request, listener, shardsIts, timeProvider, clusterState, task, - new CanMatchSearchPhaseResults(shardsIts.size()), shardsIts.size(), clusters); - this.phaseFactory = phaseFactory; + + public CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService, + BiFunction nodeIdToConnection, + Map aliasFilter, Map concreteIndexBoosts, + Executor executor, SearchRequest request, + ActionListener listener, GroupShardsIterator shardsIts, + TransportSearchAction.SearchTimeProvider timeProvider, + SearchTask task, Function, SearchPhase> phaseFactory, + SearchResponse.Clusters clusters, + CoordinatorRewriteContextProvider coordinatorRewriteContextProvider) { + super("can_match"); + this.logger = logger; + this.searchTransportService = searchTransportService; + this.nodeIdToConnection = nodeIdToConnection; + this.request = request; + this.listener = listener; this.shardsIts = shardsIts; + this.clusters = clusters; + this.timeProvider = timeProvider; + this.concreteIndexBoosts = concreteIndexBoosts; + this.aliasFilter = aliasFilter; + this.task = task; + this.phaseFactory = phaseFactory; this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; - } + this.executor = executor; + this.shardItIndexMap = new HashMap<>(); + results = new CanMatchSearchPhaseResults(shardsIts.size()); - @Override - public void addReleasable(Releasable releasable) { - throw new RuntimeException("cannot add releasable in " + getName() + " phase"); + // we compute the shard index based on the natural order of the shards + // that participate in the search request. This means that this number is + // consistent between two requests that target the same shards. + List naturalOrder = new ArrayList<>(); + shardsIts.iterator().forEachRemaining(naturalOrder::add); + CollectionUtil.timSort(naturalOrder); + for (int i = 0; i < naturalOrder.size(); i++) { + shardItIndexMap.put(naturalOrder.get(i), i); + } } @Override - protected void executePhaseOnShard(SearchShardIterator shardIt, SearchShardTarget shard, - SearchActionListener listener) { - getSearchTransport().sendCanMatch(getConnection(shard.getClusterAlias(), shard.getNodeId()), - buildShardSearchRequest(shardIt, listener.requestIndex), getTask(), listener); + public void run() throws IOException { + if (shardsIts.size() > 0) { + checkNoMissingShards(); + Version version = request.minCompatibleShardNode(); + if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { + if (checkMinimumVersion(shardsIts) == false) { + throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", + request.minCompatibleShardNode()); + } + } + + runCoordinationPhase(); + } } - @Override - protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { + private void runCoordinationPhase() { + final List matchedShardLevelRequests = new ArrayList<>(); + for (SearchShardIterator searchShardIterator : shardsIts) { + final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, + Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), + timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); + List shardSearchRequests = canMatchRequest.createShardSearchRequests(); + for (int i = 0; i < shardSearchRequests.size(); i++) { + ShardSearchRequest request = shardSearchRequests.get(i); + + boolean canMatch = true; + CoordinatorRewriteContext coordinatorRewriteContext = + coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); + if (coordinatorRewriteContext != null) { + try { + canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); + } catch (Exception e) { + // treat as if shard is still a potential match + } + } + if (canMatch) { + matchedShardLevelRequests.add(searchShardIterator); + } else { + SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); + result.setShardIndex(request.shardRequestIndex()); + results.consumeResult(result, () -> {}); + } + } + } - return phaseFactory.apply(getIterator((CanMatchSearchPhaseResults) results, shardsIts)); + if (matchedShardLevelRequests.isEmpty() == false) { + new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).start(); + } else { + finishHim(); + } } - private GroupShardsIterator getIterator(CanMatchSearchPhaseResults results, - GroupShardsIterator shardsIts) { - int cardinality = results.getNumPossibleMatches(); - FixedBitSet possibleMatches = results.getPossibleMatches(); - if (cardinality == 0) { - // this is a special case where we have no hit but we need to get at least one search response in order - // to produce a valid search result with all the aggs etc. - // Since it's possible that some of the shards that we're skipping are - // unavailable, we would try to query the node that at least has some - // shards available in order to produce a valid search result. - int shardIndexToQuery = 0; - for (int i = 0; i < shardsIts.size(); i++) { - if (shardsIts.get(i).size() > 0) { - shardIndexToQuery = i; - break; + private void checkNoMissingShards() { + assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (request.allowPartialSearchResults() == false) { + final StringBuilder missingShards = new StringBuilder(); + // Fail-fast verification of all shards being available + for (int index = 0; index < shardsIts.size(); index++) { + final SearchShardIterator shardRoutings = shardsIts.get(index); + if (shardRoutings.size() == 0) { + if (missingShards.length() > 0) { + missingShards.append(", "); + } + missingShards.append(shardRoutings.shardId()); } } - possibleMatches.set(shardIndexToQuery); + if (missingShards.length() > 0) { + //Status red - shard is missing all copies and would produce partial results for an index search + final String msg = "Search rejected due to missing shards ["+ missingShards + + "]. Consider using `allow_partial_search_results` setting to bypass this error."; + throw new SearchPhaseExecutionException(getName(), msg, null, ShardSearchFailure.EMPTY_ARRAY); + } } - SearchSourceBuilder source = getRequest().source(); - int i = 0; - for (SearchShardIterator iter : shardsIts) { - if (possibleMatches.get(i++)) { - iter.reset(); + } + + private Map> groupByNode(GroupShardsIterator shards) { + Map> requests = new HashMap<>(); + for (int i = 0; i < shards.size(); i++) { + final SearchShardIterator shardRoutings = shards.get(i); + assert shardRoutings.skip() == false; + assert shardItIndexMap.containsKey(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), + t -> new ArrayList<>()).add(shardRoutings); } else { - iter.resetAndSkip(); + requests.computeIfAbsent(new SendingTarget(null, null), + t -> new ArrayList<>()).add(shardRoutings); } } - if (shouldSortShards(results.minAndMaxes) == false) { - return shardsIts; - } - FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); - return new GroupShardsIterator<>(sortShards(shardsIts, results.minAndMaxes, fieldSort.order())); + return requests; } - @Override - protected void performPhaseOnShard(int shardIndex, SearchShardIterator shardIt, SearchShardTarget shard) { - CoordinatorRewriteContext coordinatorRewriteContext = - coordinatorRewriteContextProvider.getCoordinatorRewriteContext(shardIt.shardId().getIndex()); + class Round implements Runnable { + private final GroupShardsIterator shards; + private final CountDown countDown; + private final AtomicReferenceArray responses; - if (coordinatorRewriteContext == null) { - super.performPhaseOnShard(shardIndex, shardIt, shard); - return; + Round(GroupShardsIterator shards) { + this.shards = shards; + this.countDown = new CountDown(shards.size()); + this.responses = new AtomicReferenceArray<>(shardsIts.size()); } - try { - ShardSearchRequest request = buildShardSearchRequest(shardIt, shardIndex); - boolean canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); + public void start() { + try { + run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(CanMatchPreFilterSearchPhase.this, "", e); + } + } - // Trigger the query as there's still a chance that we can skip - // this shard given other query filters that we cannot apply - // in the coordinator - if (canMatch) { - super.performPhaseOnShard(shardIndex, shardIt, shard); - return; + + @Override + public void run() { + final Map> requests = groupByNode(shards); + + for (Map.Entry> entry : requests.entrySet()) { + CanMatchRequest canMatchRequest = createCanMatchRequest(entry); + List shardLevelRequests = canMatchRequest.getShardLevelRequests(); + + if (entry.getKey().nodeId == null) { + // no target node: just mark the requests as failed + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), null); + } + continue; + } + + try { + searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchRequest, + task, new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchResponse) { + for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { + SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); + if (response != null) { + response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + onOperation(response.getShardIndex(), response); + } + } + for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { + Exception failure = canMatchResponse.getFailures().get(i); + if (failure != null) { + onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + } + } + } + + @Override + public void onFailure(Exception e) { + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); + } + } + } + ); + } catch (Exception e) { + for (CanMatchRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); + } + } } + } + + private void onOperation(int idx, SearchService.CanMatchResponse response) { + responses.set(idx, response); + results.consumeResult(response, () -> { + if (countDown.countDown()) { + finishPhase(); + } + }); + } + + private void onOperationFailed(int idx, Exception e) { + responses.set(idx, e); + results.consumeShardFailure(idx); + if (countDown.countDown()) { + finishPhase(); + } + } - CanMatchResponse result = new CanMatchResponse(canMatch, null); - result.setSearchShardTarget(shard == null ? new SearchShardTarget(null, shardIt.shardId(), shardIt.getClusterAlias(), - shardIt.getOriginalIndices()) : shard); - result.setShardIndex(shardIndex); - fork(() -> onShardResult(result, shardIt)); + private void finishPhase() { + List remainingShards = new ArrayList<>(); + for (SearchShardIterator ssi : shards) { + int shardIndex = shardItIndexMap.get(ssi); + Object resp = responses.get(shardIndex); + if (resp instanceof Exception) { + // do something meaningful + remainingShards.add(ssi); + } + } + if (remainingShards.isEmpty()) { + finishHim(); + } else { + // trigger another round + new Round(new GroupShardsIterator<>(remainingShards)).start(); + } + } + } + + private static class SendingTarget { + @Nullable + private final String clusterAlias; + @Nullable + private final String nodeId; + + SendingTarget(@Nullable String clusterAlias, @Nullable String nodeId) { + this.clusterAlias = clusterAlias; + this.nodeId = nodeId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SendingTarget that = (SendingTarget) o; + return Objects.equals(clusterAlias, that.clusterAlias) && + Objects.equals(nodeId, that.nodeId); + } + + @Override + public int hashCode() { + return Objects.hash(clusterAlias, nodeId); + } + } + + private CanMatchRequest createCanMatchRequest(Map.Entry> entry) { + final SearchShardIterator first = entry.getValue().get(0); + final List shardLevelRequests = + entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); + assert entry.getValue().stream().allMatch(Objects::nonNull); + assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices() != null); + assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); + assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); + final CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, + shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); + return canMatchRequest; + } + + private void finishHim() { + try { + phaseFactory.apply(getIterator(results, shardsIts)).start(); } catch (Exception e) { - // If we fail to rewrite it on the coordinator, just try to execute - // the query in the shard. - super.performPhaseOnShard(shardIndex, shardIt, shard); + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(this, "", e); } } - private static List sortShards(GroupShardsIterator shardsIts, - MinAndMax[] minAndMaxes, - SortOrder order) { - return IntStream.range(0, shardsIts.size()) - .boxed() - .sorted(shardComparator(shardsIts, minAndMaxes, order)) - .map(shardsIts::get) - .collect(Collectors.toList()); + private static final float DEFAULT_INDEX_BOOST = 1.0f; + + public final CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { + AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); + assert filter != null; + float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); + CanMatchRequest.Shard shardRequest = new CanMatchRequest.Shard(shardIt.shardId(), + shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); + return shardRequest; } - private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { - Class clazz = null; - for (MinAndMax minAndMax : minAndMaxes) { - if (clazz == null) { - clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); - } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { - // we don't support sort values that mix different types (e.g.: long/double, numeric/keyword). - // TODO: we could fail the request because there is a high probability - // that the merging of topdocs will fail later for the same reason ? - return false; + private boolean checkMinimumVersion(GroupShardsIterator shardsIts) { + for (SearchShardIterator it : shardsIts) { + if (it.getTargetNodeIds().isEmpty() == false) { + boolean isCompatible = it.getTargetNodeIds().stream().anyMatch(nodeId -> { + Transport.Connection conn = getConnection(new SendingTarget(it.getClusterAlias(), nodeId)); + return conn == null ? true : conn.getVersion().onOrAfter(request.minCompatibleShardNode()); + }); + if (isCompatible == false) { + return false; + } } } - return clazz != null; + return true; } - private static Comparator shardComparator(GroupShardsIterator shardsIts, - MinAndMax[] minAndMaxes, - SortOrder order) { - final Comparator comparator = Comparator.comparing( - index -> minAndMaxes[index], - forciblyCast(MinAndMax.getComparator(order)) - ); + @Override + public void start() { + if (getNumShards() == 0) { + //no search shards to search on, bail with empty response + //(it happens with search across _all with no indices around and consistent with broadcast operations) + int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : + request.source().trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : + request.source().trackTotalHitsUpTo(); + // total hits is null in the response if the tracking of total hits is disabled + boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; + listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, + 0, timeProvider.buildTookInMillis(), ShardSearchFailure.EMPTY_ARRAY, clusters, null)); + return; + } - return comparator.thenComparing(index -> shardsIts.get(index)); + try { + run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure(this, "", e); + } } - private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { + + public final void onPhaseFailure(SearchPhase phase, String msg, Exception cause) { + listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); + } + + public final Transport.Connection getConnection(SendingTarget sendingTarget) { + Transport.Connection conn = nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId); + Version minVersion = request.minCompatibleShardNode(); + if (minVersion != null && conn != null && conn.getVersion().before(minVersion)) { + throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", minVersion); + } + return conn; + } + + private int getNumShards() { + return shardsIts.size(); + } + + private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { private final FixedBitSet possibleMatches; private final MinAndMax[] minAndMaxes; private int numPossibleMatches; @@ -207,7 +444,7 @@ private static final class CanMatchSearchPhaseResults extends SearchPhaseResults } @Override - void consumeResult(CanMatchResponse result, Runnable next) { + void consumeResult(SearchService.CanMatchResponse result, Runnable next) { try { consumeResult(result.getShardIndex(), result.canMatch(), result.estimatedMinAndMax()); } finally { @@ -243,8 +480,81 @@ synchronized FixedBitSet getPossibleMatches() { } @Override - Stream getSuccessfulResults() { + Stream getSuccessfulResults() { return Stream.empty(); } } + + private GroupShardsIterator getIterator(CanMatchSearchPhaseResults results, + GroupShardsIterator shardsIts) { + int cardinality = results.getNumPossibleMatches(); + FixedBitSet possibleMatches = results.getPossibleMatches(); + if (cardinality == 0) { + // this is a special case where we have no hit but we need to get at least one search response in order + // to produce a valid search result with all the aggs etc. + // Since it's possible that some of the shards that we're skipping are + // unavailable, we would try to query the node that at least has some + // shards available in order to produce a valid search result. + int shardIndexToQuery = 0; + for (int i = 0; i < shardsIts.size(); i++) { + if (shardsIts.get(i).size() > 0) { + shardIndexToQuery = i; + break; + } + } + possibleMatches.set(shardIndexToQuery); + } + SearchSourceBuilder source = request.source(); + int i = 0; + for (SearchShardIterator iter : shardsIts) { + if (possibleMatches.get(i++)) { + iter.reset(); + } else { + iter.resetAndSkip(); + } + } + if (shouldSortShards(results.minAndMaxes) == false) { + return shardsIts; + } + FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(source); + return new GroupShardsIterator<>(sortShards(shardsIts, results.minAndMaxes, fieldSort.order())); + } + + private static List sortShards(GroupShardsIterator shardsIts, + MinAndMax[] minAndMaxes, + SortOrder order) { + return IntStream.range(0, shardsIts.size()) + .boxed() + .sorted(shardComparator(shardsIts, minAndMaxes, order)) + .map(shardsIts::get) + .collect(Collectors.toList()); + } + + private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { + Class clazz = null; + for (MinAndMax minAndMax : minAndMaxes) { + if (clazz == null) { + clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); + } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { + // we don't support sort values that mix different types (e.g.: long/double, numeric/keyword). + // TODO: we could fail the request because there is a high probability + // that the merging of topdocs will fail later for the same reason ? + return false; + } + } + return clazz != null; + } + + private static Comparator shardComparator(GroupShardsIterator shardsIts, + MinAndMax[] minAndMaxes, + SortOrder order) { + final Comparator comparator = Comparator.comparing( + index -> minAndMaxes[index], + forciblyCast(MinAndMax.getComparator(order)) + ); + + return comparator.thenComparing(index -> shardsIts.get(index)); + } + } + diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a7c60261ce3dd..a244bf5ee9e88 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -791,7 +791,7 @@ private SearchPhase searchAsyncAction( ThreadPool threadPool, SearchResponse.Clusters clusters) { if (preFilter) { - return new CanMatchPhase(logger, searchTransportService, connectionLookup, + return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators, timeProvider, task, (iter) -> { SearchPhase action = searchAsyncAction( diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java deleted file mode 100644 index a23fba7ead913..0000000000000 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPhaseTests.java +++ /dev/null @@ -1,738 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.query.AbstractQueryBuilder; -import org.elasticsearch.index.query.BoolQueryBuilder; -import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; -import org.elasticsearch.index.query.RangeQueryBuilder; -import org.elasticsearch.index.query.TermQueryBuilder; -import org.elasticsearch.index.shard.IndexLongFieldRange; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.shard.ShardLongFieldRange; -import org.elasticsearch.search.SearchService; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.internal.AliasFilter; -import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.search.sort.MinAndMax; -import org.elasticsearch.search.sort.SortBuilders; -import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.transport.Transport; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static org.elasticsearch.action.search.SearchAsyncActionTests.getShardsIter; -import static org.elasticsearch.core.Types.forciblyCast; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.mockito.Mockito.mock; - -public class CanMatchPhaseTests extends ESTestCase { - - private final CoordinatorRewriteContextProvider EMPTY_CONTEXT_PROVIDER = new StaticCoordinatorRewriteContextProviderBuilder().build(); - - public void testFilterShards() throws InterruptedException { - - final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), - System::nanoTime); - - Map lookup = new ConcurrentHashMap<>(); - DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = randomBoolean() ? null : - new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - final boolean shard1 = randomBoolean(); - final boolean shard2 = randomBoolean(); - - final AtomicInteger numRequests = new AtomicInteger(); - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - numRequests.incrementAndGet(); - final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - responses.add(new SearchService.CanMatchResponse(shard.shardId().id() == 0 ? shard1 : - shard2, null)); - } - - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); - } - }; - - AtomicReference> result = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - GroupShardsIterator shardsIter = getShardsIter("idx", - new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), - 2, randomBoolean(), primaryNode, replicaNode); - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.allowPartialSearchResults(true); - - CanMatchPhase canMatchPhase = new CanMatchPhase(logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider,null, - (iter) -> new SearchPhase("test") { - @Override - public void run() throws IOException { - result.set(iter); - latch.countDown(); - }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); - - canMatchPhase.start(); - latch.await(); - - assertThat(numRequests.get(), replicaNode == null ? equalTo(1) : lessThanOrEqualTo(2)); - - if (shard1 && shard2) { - for (SearchShardIterator i : result.get()) { - assertFalse(i.skip()); - } - } else if (shard1 == false && shard2 == false) { - assertFalse(result.get().get(0).skip()); - assertTrue(result.get().get(1).skip()); - } else { - assertEquals(0, result.get().get(0).shardId().id()); - assertEquals(1, result.get().get(1).shardId().id()); - assertEquals(shard1, result.get().get(0).skip() == false); - assertEquals(shard2, result.get().get(1).skip() == false); - } - } - - public void testFilterWithFailure() throws InterruptedException { - final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), - System::nanoTime); - Map lookup = new ConcurrentHashMap<>(); - DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - final boolean shard1 = randomBoolean(); - final boolean useReplicas = randomBoolean(); - final boolean fullFailure = randomBoolean(); - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - if (fullFailure && randomBoolean()) { - throw new IllegalArgumentException("boom"); - } - final SearchService.CanMatchResponse[] responses = - new SearchService.CanMatchResponse[request.getShardLevelRequests().size()]; - final Exception[] failures = new Exception[request.getShardLevelRequests().size()]; - for (int i = 0; i < request.getShardLevelRequests().size(); i++) { - CanMatchRequest.Shard shard = request.getShardLevelRequests().get(i); - boolean throwException = shard.shardId().id() != 0; - if (throwException) { - failures[i] = new NullPointerException(); - } else { - responses[i] = new SearchService.CanMatchResponse(shard1, null); - } - } - - new Thread(() -> { - if (fullFailure) { - listener.onFailure(new NullPointerException()); - } else { - listener.onResponse(new CanMatchNodeResponse(Arrays.asList(responses), Arrays.asList(failures))); - } - }).start(); - } - }; - - AtomicReference> result = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - GroupShardsIterator shardsIter = getShardsIter("idx", - new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), - 2, useReplicas, primaryNode, replicaNode); - - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.allowPartialSearchResults(true); - - CanMatchPhase canMatchPhase = new CanMatchPhase(logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider,null, - (iter) -> new SearchPhase("test") { - @Override - public void run() throws IOException { - result.set(iter); - latch.countDown(); - }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); - - canMatchPhase.start(); - latch.await(); - - assertEquals(0, result.get().get(0).shardId().id()); - assertEquals(1, result.get().get(1).shardId().id()); - if (fullFailure) { - assertFalse(result.get().get(0).skip()); // never skip the failure - } else { - assertEquals(shard1, result.get().get(0).skip() == false); - } - assertFalse(result.get().get(1).skip()); // never skip the failure - } - - public void testSortShards() throws InterruptedException { - final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), - System::nanoTime); - - Map lookup = new ConcurrentHashMap<>(); - DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - - for (SortOrder order : SortOrder.values()) { - List shardIds = new ArrayList<>(); - List> minAndMaxes = new ArrayList<>(); - Set shardToSkip = new HashSet<>(); - - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - Long min = rarely() ? null : randomLong(); - Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); - MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); - boolean canMatch = frequently(); - synchronized (shardIds) { - shardIds.add(shard.shardId()); - minAndMaxes.add(minMax); - if (canMatch == false) { - shardToSkip.add(shard.shardId()); - } - } - - responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); - } - - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); - } - }; - - AtomicReference> result = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - GroupShardsIterator shardsIter = getShardsIter("logs", - new OriginalIndices(new String[]{"logs"}, SearchRequest.DEFAULT_INDICES_OPTIONS), - randomIntBetween(2, 20), randomBoolean(), primaryNode, replicaNode); - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); - searchRequest.allowPartialSearchResults(true); - - CanMatchPhase canMatchPhase = new CanMatchPhase(logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, null, - (iter) -> new SearchPhase("test") { - @Override - public void run() { - result.set(iter); - latch.countDown(); - } - }, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); - - canMatchPhase.start(); - latch.await(); - ShardId[] expected = IntStream.range(0, shardIds.size()) - .boxed() - .sorted(Comparator.comparing(minAndMaxes::get, forciblyCast(MinAndMax.getComparator(order))).thenComparing(shardIds::get)) - .map(shardIds::get) - .toArray(ShardId[]::new); - if (shardToSkip.size() == expected.length) { - // we need at least one shard to produce the empty result for aggs - shardToSkip.remove(new ShardId("logs", "_na_", 0)); - } - int pos = 0; - for (SearchShardIterator i : result.get()) { - assertEquals(shardToSkip.contains(i.shardId()), i.skip()); - assertEquals(expected[pos++], i.shardId()); - } - } - } - - public void testInvalidSortShards() throws InterruptedException { - final TransportSearchAction.SearchTimeProvider timeProvider = - new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); - - Map lookup = new ConcurrentHashMap<>(); - DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - - for (SortOrder order : SortOrder.values()) { - int numShards = randomIntBetween(2, 20); - List shardIds = new ArrayList<>(); - Set shardToSkip = new HashSet<>(); - - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - final MinAndMax minMax; - if (shard.shardId().id() == numShards-1) { - minMax = new MinAndMax<>(new BytesRef("bar"), new BytesRef("baz")); - } else { - Long min = randomLong(); - Long max = randomLongBetween(min, Long.MAX_VALUE); - minMax = new MinAndMax<>(min, max); - } - boolean canMatch = frequently(); - synchronized (shardIds) { - shardIds.add(shard.shardId()); - if (canMatch == false) { - shardToSkip.add(shard.shardId()); - } - } - responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); - } - - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); - } - }; - - AtomicReference> result = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - GroupShardsIterator shardsIter = getShardsIter("logs", - new OriginalIndices(new String[]{"logs"}, SearchRequest.DEFAULT_INDICES_OPTIONS), - numShards, randomBoolean(), primaryNode, replicaNode); - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); - searchRequest.allowPartialSearchResults(true); - - CanMatchPhase canMatchPhase = new CanMatchPhase(logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, null, - (iter) -> new SearchPhase("test") { - @Override - public void run() { - result.set(iter); - latch.countDown(); - } - }, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); - - canMatchPhase.start(); - latch.await(); - int shardId = 0; - for (SearchShardIterator i : result.get()) { - assertThat(i.shardId().id(), equalTo(shardId++)); - assertEquals(shardToSkip.contains(i.shardId()), i.skip()); - } - assertThat(result.get().size(), equalTo(numShards)); - } - } - - public void testCanMatchFilteringOnCoordinatorThatCanBeSkipped() throws Exception { - Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); - Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); - DataStream dataStream = - new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); - - List regularIndices = - randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); - - long indexMinTimestamp = randomLongBetween(0, 5000); - long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2); - StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); - String timestampFieldName = dataStream.getTimeStampField().getName(); - for (Index dataStreamIndex : dataStream.getIndices()) { - contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); - } - - RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); - // We query a range outside of the timestamp range covered by both datastream indices - rangeQueryBuilder - .from(indexMaxTimestamp + 1) - .to(indexMaxTimestamp + 2); - - BoolQueryBuilder queryBuilder = new BoolQueryBuilder() - .filter(rangeQueryBuilder); - - if (randomBoolean()) { - // Add an additional filter that cannot be evaluated in the coordinator but shouldn't - // affect the end result as we're filtering - queryBuilder.filter(new TermQueryBuilder("fake", "value")); - } - - assignShardsAndExecuteCanMatchPhase(dataStream, regularIndices, contextProviderBuilder.build(), queryBuilder, - (updatedSearchShardIterators, requests) -> { - List skippedShards = updatedSearchShardIterators.stream() - .filter(SearchShardIterator::skip) - .collect(Collectors.toList());; - - List nonSkippedShards = updatedSearchShardIterators.stream() - .filter(searchShardIterator -> searchShardIterator.skip() == false) - .collect(Collectors.toList());; - - int regularIndexShardCount = (int) updatedSearchShardIterators.stream() - .filter(s -> regularIndices.contains(s.shardId().getIndex())) - .count(); - - // When all the shards can be skipped we should query at least 1 - // in order to get a valid search response. - if (regularIndexShardCount == 0) { - assertThat(nonSkippedShards.size(), equalTo(1)); - } else { - boolean allNonSkippedShardsAreFromRegularIndices = nonSkippedShards.stream() - .allMatch(shardIterator -> regularIndices.contains(shardIterator.shardId().getIndex())); - - assertThat(allNonSkippedShardsAreFromRegularIndices, equalTo(true)); - } - - boolean allSkippedShardAreFromDataStream = skippedShards.stream() - .allMatch(shardIterator -> dataStream.getIndices().contains(shardIterator.shardId().getIndex())); - assertThat(allSkippedShardAreFromDataStream, equalTo(true)); - - boolean allRequestsWereTriggeredAgainstRegularIndices = requests.stream() - .allMatch(request -> regularIndices.contains(request.shardId().getIndex())); - assertThat(allRequestsWereTriggeredAgainstRegularIndices, equalTo(true)); - }); - } - - public void testCanMatchFilteringOnCoordinatorParsingFails() throws Exception { - Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); - Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); - DataStream dataStream = - new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); - - List regularIndices = - randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); - - long indexMinTimestamp = randomLongBetween(0, 5000); - long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2); - StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); - String timestampFieldName = dataStream.getTimeStampField().getName(); - for (Index dataStreamIndex : dataStream.getIndices()) { - contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); - } - - RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); - // Query with a non default date format - rangeQueryBuilder - .from("2020-1-01") - .to("2021-1-01"); - - BoolQueryBuilder queryBuilder = new BoolQueryBuilder() - .filter(rangeQueryBuilder); - - if (randomBoolean()) { - // Add an additional filter that cannot be evaluated in the coordinator but shouldn't - // affect the end result as we're filtering - queryBuilder.filter(new TermQueryBuilder("fake", "value")); - } - - assignShardsAndExecuteCanMatchPhase(dataStream, - regularIndices, - contextProviderBuilder.build(), - queryBuilder, - this::assertAllShardsAreQueried - ); - } - - public void testCanMatchFilteringOnCoordinatorThatCanNotBeSkipped() throws Exception { - // Generate indices - Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID()); - Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID()); - DataStream dataStream = - new DataStream("mydata", new DataStream.TimestampField("@timestamp"), List.of(dataStreamIndex1, dataStreamIndex2)); - - List regularIndices = - randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID())); - - long indexMinTimestamp = 10; - long indexMaxTimestamp = 20; - StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder(); - String timestampFieldName = dataStream.getTimeStampField().getName(); - for (Index dataStreamIndex : dataStream.getIndices()) { - contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampFieldName, indexMinTimestamp, indexMaxTimestamp); - } - - BoolQueryBuilder queryBuilder = new BoolQueryBuilder(); - // Query inside of the data stream index range - if (randomBoolean()) { - // Query generation - RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName); - // We query a range within the timestamp range covered by both datastream indices - rangeQueryBuilder - .from(indexMinTimestamp) - .to(indexMaxTimestamp); - - queryBuilder.filter(rangeQueryBuilder); - - if (randomBoolean()) { - // Add an additional filter that cannot be evaluated in the coordinator but shouldn't - // affect the end result as we're filtering - queryBuilder.filter(new TermQueryBuilder("fake", "value")); - } - } else { - // We query a range outside of the timestamp range covered by both datastream indices - RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampFieldName) - .from(indexMaxTimestamp + 1) - .to(indexMaxTimestamp + 2); - - TermQueryBuilder termQueryBuilder = new TermQueryBuilder("fake", "value"); - - // This is always evaluated as true in the coordinator as we cannot determine there if - // the term query clause is false. - queryBuilder.should(rangeQueryBuilder) - .should(termQueryBuilder); - } - - assignShardsAndExecuteCanMatchPhase(dataStream, - regularIndices, - contextProviderBuilder.build(), - queryBuilder, - this::assertAllShardsAreQueried - ); - } - - private void assertAllShardsAreQueried(List updatedSearchShardIterators, List requests) { - int skippedShards = (int) updatedSearchShardIterators.stream() - .filter(SearchShardIterator::skip) - .count(); - - assertThat(skippedShards, equalTo(0)); - - int nonSkippedShards = (int) updatedSearchShardIterators.stream() - .filter(searchShardIterator -> searchShardIterator.skip() == false) - .count(); - - assertThat(nonSkippedShards, equalTo(updatedSearchShardIterators.size())); - - int shardsWithPrimariesAssigned = (int) updatedSearchShardIterators.stream() - .filter(s -> s.size() > 0) - .count(); - assertThat(requests.size(), equalTo(shardsWithPrimariesAssigned)); - } - - private > - void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, - List regularIndices, - CoordinatorRewriteContextProvider contextProvider, - AbstractQueryBuilder query, - BiConsumer, - List> canMatchResultsConsumer) throws Exception { - Map lookup = new ConcurrentHashMap<>(); - DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - - List indicesToSearch = new ArrayList<>(); - indicesToSearch.add(dataStream.getName()); - for (Index regularIndex : regularIndices) { - indicesToSearch.add(regularIndex.getName()); - } - - String[] indices = indicesToSearch.toArray(new String[0]); - OriginalIndices originalIndices = new OriginalIndices(indices, SearchRequest.DEFAULT_INDICES_OPTIONS); - - boolean atLeastOnePrimaryAssigned = false; - final List originalShardIters = new ArrayList<>(); - for (Index dataStreamIndex : dataStream.getIndices()) { - // If we have to execute the can match request against all the shards - // and none is assigned, the phase is considered as failed meaning that the next phase won't be executed - boolean withAssignedPrimaries = randomBoolean() || atLeastOnePrimaryAssigned == false; - int numShards = randomIntBetween(1, 6); - originalShardIters.addAll( - getShardsIter(dataStreamIndex, - originalIndices, - numShards, - false, - withAssignedPrimaries ? primaryNode : null, - null) - ); - atLeastOnePrimaryAssigned |= withAssignedPrimaries; - } - - for (Index regularIndex : regularIndices) { - originalShardIters.addAll( - getShardsIter(regularIndex, - originalIndices, - randomIntBetween(1, 6), - randomBoolean(), - primaryNode, - replicaNode) - ); - } - GroupShardsIterator shardsIter = GroupShardsIterator.sortAndCreate(originalShardIters); - - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.indices(indices); - searchRequest.allowPartialSearchResults(true); - - final AliasFilter aliasFilter; - if (randomBoolean()) { - // Apply the query on the request body - SearchSourceBuilder searchSourceBuilder = SearchSourceBuilder.searchSource(); - searchSourceBuilder.query(query); - searchRequest.source(searchSourceBuilder); - - // Sometimes apply the same query in the alias filter too - aliasFilter = new AliasFilter(randomBoolean() ? query : null, Strings.EMPTY_ARRAY); - } else { - // Apply the query as an alias filter - aliasFilter = new AliasFilter(query, Strings.EMPTY_ARRAY); - } - - Map aliasFilters = new HashMap<>(); - for (Index dataStreamIndex : dataStream.getIndices()) { - aliasFilters.put(dataStreamIndex.getUUID(), aliasFilter); - } - - for (Index regularIndex : regularIndices) { - aliasFilters.put(regularIndex.getUUID(), aliasFilter); - } - - // We respond by default that the query can match - final List requests = Collections.synchronizedList(new ArrayList<>()); - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - requests.add(request.createShardSearchRequest(shard)); - responses.add(new SearchService.CanMatchResponse(true, null)); - } - - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); - } - }; - - final TransportSearchAction.SearchTimeProvider timeProvider = - new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); - - AtomicReference> result = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - CanMatchPhase canMatchPhase = new CanMatchPhase(logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - aliasFilters, - Collections.emptyMap(), - EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, - null, - shardsIter, - timeProvider, - null, - (iter) -> new SearchPhase("test") { - @Override - public void run() throws IOException { - result.set(iter); - latch.countDown(); - } - }, - SearchResponse.Clusters.EMPTY, - contextProvider); - - canMatchPhase.start(); - latch.await(); - - List updatedSearchShardIterators = new ArrayList<>(); - for (SearchShardIterator updatedSearchShardIterator : result.get()) { - updatedSearchShardIterators.add(updatedSearchShardIterator); - } - - canMatchResultsConsumer.accept(updatedSearchShardIterators, requests); - } - - private static class StaticCoordinatorRewriteContextProviderBuilder { - private ClusterState clusterState = ClusterState.EMPTY_STATE; - private final Map fields = new HashMap<>(); - - private void addIndexMinMaxTimestamps(Index index, String fieldName, long minTimeStamp, long maxTimestamp) { - if (clusterState.metadata().index(index) != null) { - throw new IllegalArgumentException("Min/Max timestamps for " + index + " were already defined"); - } - - IndexLongFieldRange timestampRange = IndexLongFieldRange.NO_SHARDS - .extendWithShardRange(0, 1, ShardLongFieldRange.of(minTimeStamp, maxTimestamp)); - - Settings.Builder indexSettings = settings(Version.CURRENT) - .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()); - - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(index.getName()) - .settings(indexSettings) - .numberOfShards(1) - .numberOfReplicas(0) - .timestampRange(timestampRange); - - Metadata.Builder metadataBuilder = - Metadata.builder(clusterState.metadata()) - .put(indexMetadataBuilder); - - clusterState = ClusterState.builder(clusterState) - .metadata(metadataBuilder) - .build(); - - fields.put(index, new DateFieldMapper.DateFieldType(fieldName)); - } - - public CoordinatorRewriteContextProvider build() { - return new CoordinatorRewriteContextProvider(NamedXContentRegistry.EMPTY, - mock(NamedWriteableRegistry.class), - mock(Client.class), - System::currentTimeMillis, - () -> clusterState, - fields::get); - } - } -} diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 0ae9d743e6143..008033790271d 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -5,6 +5,7 @@ * in compliance with, at your election, the Elastic License 2.0 or the Server * Side Public License, v 1. */ + package org.elasticsearch.action.search; import org.apache.lucene.util.BytesRef; @@ -34,9 +35,7 @@ import org.elasticsearch.index.shard.IndexLongFieldRange; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardLongFieldRange; -import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; -import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchRequest; @@ -48,6 +47,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -57,8 +57,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -67,6 +66,7 @@ import static org.elasticsearch.action.search.SearchAsyncActionTests.getShardsIter; import static org.elasticsearch.core.Types.forciblyCast; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.mockito.Mockito.mock; public class CanMatchPreFilterSearchPhaseTests extends ESTestCase { @@ -80,18 +80,26 @@ public void testFilterShards() throws InterruptedException { Map lookup = new ConcurrentHashMap<>(); DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode replicaNode = randomBoolean() ? null : + new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); final boolean shard1 = randomBoolean(); final boolean shard2 = randomBoolean(); + final AtomicInteger numRequests = new AtomicInteger(); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, ShardSearchRequest request, SearchTask task, - ActionListener listener) { - new Thread(() -> listener.onResponse(new SearchService.CanMatchResponse(request.shardId().id() == 0 ? shard1 : - shard2, null))).start(); + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + numRequests.incrementAndGet(); + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + responses.add(new SearchService.CanMatchResponse(shard.shardId().id() == 0 ? shard1 : + shard2, null)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); } }; @@ -108,17 +116,19 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, ClusterState.EMPTY_STATE, null, + searchRequest, null, shardsIter, timeProvider,null, (iter) -> new SearchPhase("test") { - @Override - public void run() throws IOException { - result.set(iter); - latch.countDown(); - }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); + @Override + public void run() throws IOException { + result.set(iter); + latch.countDown(); + }}, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); canMatchPhase.start(); latch.await(); + assertThat(numRequests.get(), replicaNode == null ? equalTo(1) : lessThanOrEqualTo(2)); + if (shard1 && shard2) { for (SearchShardIterator i : result.get()) { assertFalse(i.skip()); @@ -143,22 +153,35 @@ public void testFilterWithFailure() throws InterruptedException { lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); final boolean shard1 = randomBoolean(); + final boolean useReplicas = randomBoolean(); + final boolean fullFailure = randomBoolean(); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, ShardSearchRequest request, SearchTask task, - ActionListener listener) { - boolean throwException = request.shardId().id() != 0; - if (throwException && randomBoolean()) { + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + if (fullFailure && randomBoolean()) { throw new IllegalArgumentException("boom"); - } else { - new Thread(() -> { - if (throwException == false) { - listener.onResponse(new SearchService.CanMatchResponse(shard1, null)); - } else { - listener.onFailure(new NullPointerException()); - } - }).start(); } + final SearchService.CanMatchResponse[] responses = + new SearchService.CanMatchResponse[request.getShardLevelRequests().size()]; + final Exception[] failures = new Exception[request.getShardLevelRequests().size()]; + for (int i = 0; i < request.getShardLevelRequests().size(); i++) { + CanMatchRequest.Shard shard = request.getShardLevelRequests().get(i); + boolean throwException = shard.shardId().id() != 0; + if (throwException) { + failures[i] = new NullPointerException(); + } else { + responses[i] = new SearchService.CanMatchResponse(shard1, null); + } + } + + new Thread(() -> { + if (fullFailure) { + listener.onFailure(new NullPointerException()); + } else { + listener.onResponse(new CanMatchNodeResponse(Arrays.asList(responses), Arrays.asList(failures))); + } + }).start(); } }; @@ -166,7 +189,7 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req CountDownLatch latch = new CountDownLatch(1); GroupShardsIterator shardsIter = getShardsIter("idx", new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS), - 2, randomBoolean(), primaryNode, replicaNode); + 2, useReplicas, primaryNode, replicaNode); final SearchRequest searchRequest = new SearchRequest(); searchRequest.allowPartialSearchResults(true); @@ -176,7 +199,7 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, ClusterState.EMPTY_STATE, null, + searchRequest, null, shardsIter, timeProvider,null, (iter) -> new SearchPhase("test") { @Override public void run() throws IOException { @@ -189,110 +212,14 @@ public void run() throws IOException { assertEquals(0, result.get().get(0).shardId().id()); assertEquals(1, result.get().get(1).shardId().id()); - assertEquals(shard1, result.get().get(0).skip() == false); + if (fullFailure) { + assertFalse(result.get().get(0).skip()); // never skip the failure + } else { + assertEquals(shard1, result.get().get(0).skip() == false); + } assertFalse(result.get().get(1).skip()); // never skip the failure } - /* - * In cases that a query coordinating node held all the shards for a query, the can match phase would recurse and end in stack overflow - * when subjected to max concurrent search requests. This test is a test for that situation. - */ - public void testLotsOfShards() throws InterruptedException { - final TransportSearchAction.SearchTimeProvider timeProvider = - new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); - - final Map lookup = new ConcurrentHashMap<>(); - final DiscoveryNode primaryNode = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Version.CURRENT); - final DiscoveryNode replicaNode = new DiscoveryNode("node_2", buildNewFakeTransportAddress(), Version.CURRENT); - lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode)); - lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode)); - - - final SearchTransportService searchTransportService = - new SearchTransportService(null, null, null) { - @Override - public void sendCanMatch( - Transport.Connection connection, - ShardSearchRequest request, - SearchTask task, - ActionListener listener) { - listener.onResponse(new SearchService.CanMatchResponse(randomBoolean(), null)); - } - }; - - final CountDownLatch latch = new CountDownLatch(1); - final OriginalIndices originalIndices = new OriginalIndices(new String[]{"idx"}, SearchRequest.DEFAULT_INDICES_OPTIONS); - final GroupShardsIterator shardsIter = - getShardsIter("idx", originalIndices, 4096, randomBoolean(), primaryNode, replicaNode); - final ExecutorService executor = Executors.newFixedThreadPool(randomIntBetween(1, Runtime.getRuntime().availableProcessors())); - final SearchRequest searchRequest = new SearchRequest(); - searchRequest.allowPartialSearchResults(true); - SearchTransportService transportService = new SearchTransportService(null, null, null); - ActionListener responseListener = ActionListener.wrap(response -> {}, - (e) -> { throw new AssertionError("unexpected", e);}); - Map aliasFilters = Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)); - final CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( - logger, - searchTransportService, - (clusterAlias, node) -> lookup.get(node), - Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), - EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, - null, - shardsIter, - timeProvider, - ClusterState.EMPTY_STATE, - null, - (iter) -> new AbstractSearchAsyncAction<>( - "test", - logger, - transportService, - (cluster, node) -> { - assert cluster == null : "cluster was not null: " + cluster; - return lookup.get(node); - }, - aliasFilters, - Collections.emptyMap(), - executor, - searchRequest, - responseListener, - iter, - new TransportSearchAction.SearchTimeProvider(0, 0, () -> 0), - ClusterState.EMPTY_STATE, - null, - new ArraySearchPhaseResults<>(iter.size()), - randomIntBetween(1, 32), - SearchResponse.Clusters.EMPTY) { - - @Override - protected SearchPhase getNextPhase(SearchPhaseResults results, SearchPhaseContext context) { - return new SearchPhase("test") { - @Override - public void run() { - latch.countDown(); - } - }; - } - - @Override - protected void executePhaseOnShard( - final SearchShardIterator shardIt, - final SearchShardTarget shard, - final SearchActionListener listener) { - if (randomBoolean()) { - listener.onResponse(new SearchPhaseResult() {}); - } else { - listener.onFailure(new Exception("failure")); - } - } - }, SearchResponse.Clusters.EMPTY, EMPTY_CONTEXT_PROVIDER); - - canMatchPhase.start(); - latch.await(); - executor.shutdown(); - } - public void testSortShards() throws InterruptedException { final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), System::nanoTime); @@ -310,20 +237,26 @@ public void testSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, ShardSearchRequest request, SearchTask task, - ActionListener listener) { - Long min = rarely() ? null : randomLong(); - Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); - MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); - boolean canMatch = frequently(); - synchronized (shardIds) { - shardIds.add(request.shardId()); - minAndMaxes.add(minMax); - if (canMatch == false) { - shardToSkip.add(request.shardId()); + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + Long min = rarely() ? null : randomLong(); + Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); + MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); + boolean canMatch = frequently(); + synchronized (shardIds) { + shardIds.add(shard.shardId()); + minAndMaxes.add(minMax); + if (canMatch == false) { + shardToSkip.add(shard.shardId()); + } } + + responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); } - new Thread(() -> listener.onResponse(new SearchService.CanMatchResponse(canMatch, minMax))).start(); + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); } }; @@ -341,7 +274,7 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, ClusterState.EMPTY_STATE, null, + searchRequest, null, shardsIter, timeProvider, null, (iter) -> new SearchPhase("test") { @Override public void run() { @@ -386,24 +319,29 @@ public void testInvalidSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, ShardSearchRequest request, SearchTask task, - ActionListener listener) { - final MinAndMax minMax; - if (request.shardId().id() == numShards-1) { - minMax = new MinAndMax<>(new BytesRef("bar"), new BytesRef("baz")); - } else { - Long min = randomLong(); - Long max = randomLongBetween(min, Long.MAX_VALUE); - minMax = new MinAndMax<>(min, max); - } - boolean canMatch = frequently(); - synchronized (shardIds) { - shardIds.add(request.shardId()); - if (canMatch == false) { - shardToSkip.add(request.shardId()); + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + final MinAndMax minMax; + if (shard.shardId().id() == numShards-1) { + minMax = new MinAndMax<>(new BytesRef("bar"), new BytesRef("baz")); + } else { + Long min = randomLong(); + Long max = randomLongBetween(min, Long.MAX_VALUE); + minMax = new MinAndMax<>(min, max); } + boolean canMatch = frequently(); + synchronized (shardIds) { + shardIds.add(shard.shardId()); + if (canMatch == false) { + shardToSkip.add(shard.shardId()); + } + } + responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); } - new Thread(() -> listener.onResponse(new SearchService.CanMatchResponse(canMatch, minMax))).start(); + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); } }; @@ -421,7 +359,7 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, - searchRequest, null, shardsIter, timeProvider, ClusterState.EMPTY_STATE, null, + searchRequest, null, shardsIter, timeProvider, null, (iter) -> new SearchPhase("test") { @Override public void run() { @@ -707,10 +645,15 @@ void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, final List requests = Collections.synchronizedList(new ArrayList<>()); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, ShardSearchRequest request, SearchTask task, - ActionListener listener) { - requests.add(request); - listener.onResponse(new SearchService.CanMatchResponse(true, null)); + public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, + ActionListener listener) { + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + requests.add(request.createShardSearchRequest(shard)); + responses.add(new SearchService.CanMatchResponse(true, null)); + } + + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); } }; @@ -729,7 +672,6 @@ public void sendCanMatch(Transport.Connection connection, ShardSearchRequest req null, shardsIter, timeProvider, - ClusterState.EMPTY_STATE, null, (iter) -> new SearchPhase("test") { @Override From 75e2bb5291143f9ebf9cadad3f2fdc9d3c7120b0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 12:25:24 +0200 Subject: [PATCH 09/28] more renaming --- .../search/CanMatchPreFilterSearchPhase.java | 15 +++-- ...odeResponse.java => CanMatchResponse.java} | 15 ++--- .../action/search/SearchTransportService.java | 23 +++---- .../search/CanMatchShardResponse.java | 52 +++++++++++++++ .../elasticsearch/search/SearchService.java | 66 ++++--------------- .../CanMatchPreFilterSearchPhaseTests.java | 44 ++++++------- 6 files changed, 114 insertions(+), 101 deletions(-) rename server/src/main/java/org/elasticsearch/action/search/{CanMatchNodeResponse.java => CanMatchResponse.java} (67%) create mode 100644 server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 0d66356ba6226..40b565a887863 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -19,6 +19,7 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.index.query.CoordinatorRewriteContext; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; +import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -147,7 +148,7 @@ private void runCoordinationPhase() { if (canMatch) { matchedShardLevelRequests.add(searchShardIterator); } else { - SearchService.CanMatchResponse result = new SearchService.CanMatchResponse(canMatch, null); + CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); result.setShardIndex(request.shardRequestIndex()); results.consumeResult(result, () -> {}); } @@ -245,9 +246,9 @@ public void run() { searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchRequest, task, new ActionListener<>() { @Override - public void onResponse(CanMatchNodeResponse canMatchResponse) { + public void onResponse(CanMatchResponse canMatchResponse) { for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - SearchService.CanMatchResponse response = canMatchResponse.getResponses().get(i); + CanMatchShardResponse response = canMatchResponse.getResponses().get(i); if (response != null) { response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); onOperation(response.getShardIndex(), response); @@ -277,7 +278,7 @@ public void onFailure(Exception e) { } } - private void onOperation(int idx, SearchService.CanMatchResponse response) { + private void onOperation(int idx, CanMatchShardResponse response) { responses.set(idx, response); results.consumeResult(response, () -> { if (countDown.countDown()) { @@ -432,7 +433,7 @@ private int getNumShards() { return shardsIts.size(); } - private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { + private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { private final FixedBitSet possibleMatches; private final MinAndMax[] minAndMaxes; private int numPossibleMatches; @@ -444,7 +445,7 @@ private static final class CanMatchSearchPhaseResults extends SearchPhaseResults } @Override - void consumeResult(SearchService.CanMatchResponse result, Runnable next) { + void consumeResult(CanMatchShardResponse result, Runnable next) { try { consumeResult(result.getShardIndex(), result.canMatch(), result.estimatedMinAndMax()); } finally { @@ -480,7 +481,7 @@ synchronized FixedBitSet getPossibleMatches() { } @Override - Stream getSuccessfulResults() { + Stream getSuccessfulResults() { return Stream.empty(); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java similarity index 67% rename from server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java rename to server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java index 9eee993f8c3ca..171fb97625ca8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java @@ -8,27 +8,26 @@ package org.elasticsearch.action.search; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.transport.TransportResponse; import java.io.IOException; import java.util.List; -public class CanMatchNodeResponse extends TransportResponse { +public class CanMatchResponse extends TransportResponse { - private final List responses; + private final List responses; private final List failures; - public CanMatchNodeResponse(StreamInput in) throws IOException { + public CanMatchResponse(StreamInput in) throws IOException { super(in); - responses = in.readList(i -> i.readOptionalWriteable(SearchService.CanMatchResponse::new)); + responses = in.readList(i -> i.readOptionalWriteable(CanMatchShardResponse::new)); failures = in.readList(StreamInput::readException); } - public CanMatchNodeResponse(List responses, List failures) { + public CanMatchResponse(List responses, List failures) { this.responses = responses; this.failures = failures; } @@ -39,7 +38,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(failures, StreamOutput::writeException); } - public List getResponses() { + public List getResponses() { return responses; } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 2dfdf4e1bfcee..eb6a1312126a9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.dfs.DfsSearchResult; @@ -123,18 +124,18 @@ public void sendFreeContext(Transport.Connection connection, ShardSearchContextI } public void sendCanMatch(Transport.Connection connection, final ShardSearchRequest request, SearchTask task, final - ActionListener listener) { + ActionListener listener) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NAME, request, task, - TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, SearchService.CanMatchResponse::new)); + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchShardResponse::new)); } public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final - ActionListener listener) { + ActionListener listener) { // TODO: use minNodeVersion here to check (i.e. min{connection.getVersion(), targetNode.getVersion()}) if (connection.getVersion().onOrAfter(Version.V_8_0_0) && connection.getNode().getVersion().onOrAfter(Version.V_8_0_0)) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, - TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchResponse::new)); } else { // BWC layer: translate into shard-level requests final List shardSearchRequests = request.createShardSearchRequests(); @@ -145,7 +146,7 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest int finalI = i; sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { @Override - public void onResponse(SearchService.CanMatchResponse response) { + public void onResponse(CanMatchShardResponse response) { results.set(finalI, response); maybeFinish(); } @@ -158,18 +159,18 @@ public void onFailure(Exception e) { private void maybeFinish() { if (counter.incrementAndGet() == shardSearchRequests.size()) { - List responses = new ArrayList<>(shardSearchRequests.size()); + List responses = new ArrayList<>(shardSearchRequests.size()); List failures = new ArrayList<>(shardSearchRequests.size()); for (int i = 0; i < results.length(); i++) { Object o = results.get(i); - if (o instanceof SearchService.CanMatchResponse) { - responses.add(i, (SearchService.CanMatchResponse) o); + if (o instanceof CanMatchShardResponse) { + responses.add(i, (CanMatchShardResponse) o); } else { assert o instanceof Exception; failures.add(i, (Exception) o); } } - CanMatchNodeResponse response = new CanMatchNodeResponse(responses, failures); + CanMatchResponse response = new CanMatchResponse(responses, failures); listener.onResponse(response); } } @@ -419,12 +420,12 @@ public static void registerRequestHandler(TransportService transportService, Sea (request, channel, task) -> { searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); - TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, true, SearchService.CanMatchResponse::new); + TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, true, CanMatchShardResponse::new); transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SAME, CanMatchRequest::new, (request, channel, task) -> { searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); - TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NODE_NAME, true, CanMatchNodeResponse::new); + TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NODE_NAME, true, CanMatchResponse::new); } diff --git a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java new file mode 100644 index 0000000000000..e1dd84635445f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java @@ -0,0 +1,52 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search; + +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.sort.MinAndMax; + +import java.io.IOException; + +public final class CanMatchShardResponse extends SearchPhaseResult { + private final boolean canMatch; + private final MinAndMax estimatedMinAndMax; + + public CanMatchShardResponse(StreamInput in) throws IOException { + super(in); + this.canMatch = in.readBoolean(); + if (in.getVersion().onOrAfter(Version.V_7_6_0)) { + estimatedMinAndMax = in.readOptionalWriteable(MinAndMax::new); + } else { + estimatedMinAndMax = null; + } + } + + public CanMatchShardResponse(boolean canMatch, MinAndMax estimatedMinAndMax) { + this.canMatch = canMatch; + this.estimatedMinAndMax = estimatedMinAndMax; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeBoolean(canMatch); + if (out.getVersion().onOrAfter(Version.V_7_6_0)) { + out.writeOptionalWriteable(estimatedMinAndMax); + } + } + + public boolean canMatch() { + return canMatch; + } + + public MinAndMax estimatedMinAndMax() { + return estimatedMinAndMax; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 334d8c28888db..7af74beec0763 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -15,12 +15,10 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.TopDocs; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.search.CanMatchNodeResponse; +import org.elasticsearch.action.search.CanMatchResponse; import org.elasticsearch.action.search.CanMatchRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchShardTask; @@ -33,8 +31,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.Releasable; @@ -412,7 +408,7 @@ public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, // check if we can shortcut the query phase entirely. if (orig.canReturnNullResponseIfMatchNoDocs()) { assert orig.scroll() == null; - final CanMatchResponse canMatchResp; + final CanMatchShardResponse canMatchResp; try { ShardSearchRequest clone = new ShardSearchRequest(orig); canMatchResp = canMatch(clone, false); @@ -420,7 +416,7 @@ public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, l.onFailure(exc); return; } - if (canMatchResp.canMatch == false) { + if (canMatchResp.canMatch() == false) { l.onResponse(QuerySearchResult.nullInstance()); return; } @@ -1219,7 +1215,7 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, Set listener) { + public void canMatch(ShardSearchRequest request, ActionListener listener) { try { listener.onResponse(canMatch(request)); } catch (IOException e) { @@ -1227,22 +1223,22 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { + public void canMatch(CanMatchRequest request, ActionListener listener) { List shardSearchRequests = request.createShardSearchRequests(); - List responses = new ArrayList<>(shardSearchRequests.size()); + List responses = new ArrayList<>(shardSearchRequests.size()); List failures = new ArrayList<>(shardSearchRequests.size()); for (int i = 0; i < shardSearchRequests.size(); i++) { ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); - CanMatchResponse canMatchResponse; + CanMatchShardResponse canMatchShardResponse; try { - canMatchResponse = canMatch(shardSearchRequest); - responses.add(i, canMatchResponse); + canMatchShardResponse = canMatch(shardSearchRequest); + responses.add(i, canMatchShardResponse); } catch (Exception e) { failures.add(i, e); } } - listener.onResponse(new CanMatchNodeResponse(responses, failures)); + listener.onResponse(new CanMatchResponse(responses, failures)); } /** @@ -1250,11 +1246,11 @@ public void canMatch(CanMatchRequest request, ActionListener estimatedMinAndMax; - - public CanMatchResponse(StreamInput in) throws IOException { - super(in); - this.canMatch = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_7_6_0)) { - estimatedMinAndMax = in.readOptionalWriteable(MinAndMax::new); - } else { - estimatedMinAndMax = null; - } - } - - public CanMatchResponse(boolean canMatch, MinAndMax estimatedMinAndMax) { - this.canMatch = canMatch; - this.estimatedMinAndMax = estimatedMinAndMax; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(canMatch); - if (out.getVersion().onOrAfter(Version.V_7_6_0)) { - out.writeOptionalWriteable(estimatedMinAndMax); - } - } - - public boolean canMatch() { - return canMatch; - } - - public MinAndMax estimatedMinAndMax() { - return estimatedMinAndMax; - } - } - /** * This helper class ensures we only execute either the success or the failure path for {@link SearchOperationListener}. * This is crucial for some implementations like {@link org.elasticsearch.index.search.stats.ShardSearchStats}. diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 008033790271d..51e0939f15af8 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -35,7 +35,7 @@ import org.elasticsearch.index.shard.IndexLongFieldRange; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardLongFieldRange; -import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchRequest; @@ -91,15 +91,15 @@ public void testFilterShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + ActionListener listener) { numRequests.incrementAndGet(); - final List responses = new ArrayList<>(); + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - responses.add(new SearchService.CanMatchResponse(shard.shardId().id() == 0 ? shard1 : + responses.add(new CanMatchShardResponse(shard.shardId().id() == 0 ? shard1 : shard2, null)); } - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); } }; @@ -158,12 +158,12 @@ public void testFilterWithFailure() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + ActionListener listener) { if (fullFailure && randomBoolean()) { throw new IllegalArgumentException("boom"); } - final SearchService.CanMatchResponse[] responses = - new SearchService.CanMatchResponse[request.getShardLevelRequests().size()]; + final CanMatchShardResponse[] responses = + new CanMatchShardResponse[request.getShardLevelRequests().size()]; final Exception[] failures = new Exception[request.getShardLevelRequests().size()]; for (int i = 0; i < request.getShardLevelRequests().size(); i++) { CanMatchRequest.Shard shard = request.getShardLevelRequests().get(i); @@ -171,7 +171,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques if (throwException) { failures[i] = new NullPointerException(); } else { - responses[i] = new SearchService.CanMatchResponse(shard1, null); + responses[i] = new CanMatchShardResponse(shard1, null); } } @@ -179,7 +179,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques if (fullFailure) { listener.onFailure(new NullPointerException()); } else { - listener.onResponse(new CanMatchNodeResponse(Arrays.asList(responses), Arrays.asList(failures))); + listener.onResponse(new CanMatchResponse(Arrays.asList(responses), Arrays.asList(failures))); } }).start(); } @@ -238,8 +238,8 @@ public void testSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); + ActionListener listener) { + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { Long min = rarely() ? null : randomLong(); Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); @@ -253,10 +253,10 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques } } - responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); + responses.add(new CanMatchShardResponse(canMatch, minMax)); } - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); } }; @@ -320,8 +320,8 @@ public void testInvalidSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); + ActionListener listener) { + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { final MinAndMax minMax; if (shard.shardId().id() == numShards-1) { @@ -338,10 +338,10 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques shardToSkip.add(shard.shardId()); } } - responses.add(new SearchService.CanMatchResponse(canMatch, minMax)); + responses.add(new CanMatchShardResponse(canMatch, minMax)); } - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); } }; @@ -646,14 +646,14 @@ void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { - final List responses = new ArrayList<>(); + ActionListener listener) { + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { requests.add(request.createShardSearchRequest(shard)); - responses.add(new SearchService.CanMatchResponse(true, null)); + responses.add(new CanMatchShardResponse(true, null)); } - new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); } }; From 44b57836df4788bc77305fad44b0745a02bf5541 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 17:43:00 +0200 Subject: [PATCH 10/28] single list --- .../search/CanMatchPreFilterSearchPhase.java | 18 ++--- .../action/search/CanMatchResponse.java | 60 +++++++++++++--- .../action/search/SearchTransportService.java | 71 ++++++++++--------- .../elasticsearch/search/SearchService.java | 15 ++-- .../CanMatchPreFilterSearchPhaseTests.java | 41 +++++------ 5 files changed, 121 insertions(+), 84 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 40b565a887863..e4e5c9d523ed1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -247,16 +247,16 @@ public void run() { task, new ActionListener<>() { @Override public void onResponse(CanMatchResponse canMatchResponse) { + assert canMatchResponse.getResponses().size() == canMatchRequest.getShardLevelRequests().size(); for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - CanMatchShardResponse response = canMatchResponse.getResponses().get(i); - if (response != null) { - response.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); - onOperation(response.getShardIndex(), response); - } - } - for (int i = 0; i < canMatchResponse.getFailures().size(); i++) { - Exception failure = canMatchResponse.getFailures().get(i); - if (failure != null) { + CanMatchResponse.ResponseOrFailure response = canMatchResponse.getResponses().get(i); + if (response.getResponse() != null) { + CanMatchShardResponse shardResponse = response.getResponse(); + shardResponse.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + onOperation(shardResponse.getShardIndex(), shardResponse); + } else { + Exception failure = response.getException(); + assert failure != null; onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java index 171fb97625ca8..8eba36b96339f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java @@ -10,6 +10,8 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.Nullable; import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.transport.TransportResponse; @@ -18,31 +20,69 @@ public class CanMatchResponse extends TransportResponse { - private final List responses; - private final List failures; + private final List responses; public CanMatchResponse(StreamInput in) throws IOException { super(in); - responses = in.readList(i -> i.readOptionalWriteable(CanMatchShardResponse::new)); - failures = in.readList(StreamInput::readException); + responses = in.readList(ResponseOrFailure::new); } - public CanMatchResponse(List responses, List failures) { + public CanMatchResponse(List responses) { this.responses = responses; - this.failures = failures; } @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(responses, StreamOutput::writeOptionalWriteable); - out.writeCollection(failures, StreamOutput::writeException); } - public List getResponses() { + public List getResponses() { return responses; } - public List getFailures() { - return failures; + public static class ResponseOrFailure implements Writeable { + + public ResponseOrFailure(CanMatchShardResponse response) { + this.response = response; + this.exception = null; + } + + public ResponseOrFailure(Exception exception) { + this.exception = exception; + this.response = null; + } + + @Nullable + public CanMatchShardResponse getResponse() { + return response; + } + + @Nullable + public Exception getException() { + return exception; + } + + private final CanMatchShardResponse response; + private final Exception exception; + + public ResponseOrFailure(StreamInput in) throws IOException { + if (in.readBoolean()) { + response = new CanMatchShardResponse(in); + exception = null; + } else { + exception = in.readException(); + response = null; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeBoolean(response != null); + if (response != null) { + response.writeTo(out); + } else { + out.writeException(exception); + } + } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index eb6a1312126a9..d5a4de1c634d9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -20,12 +20,13 @@ import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.core.Nullable; import org.elasticsearch.search.CanMatchShardResponse; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; @@ -58,7 +59,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; @@ -140,41 +140,44 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest // BWC layer: translate into shard-level requests final List shardSearchRequests = request.createShardSearchRequests(); final AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); - final AtomicInteger counter = new AtomicInteger(); - for (int i = 0; i < shardSearchRequests.size(); i++) { - ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); - int finalI = i; - sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { - @Override - public void onResponse(CanMatchShardResponse response) { - results.set(finalI, response); - maybeFinish(); - } - - @Override - public void onFailure(Exception e) { - results.set(finalI, e); - maybeFinish(); + final CountDown counter = new CountDown(shardSearchRequests.size()); + final Runnable maybeFinish = () -> { + if (counter.countDown()) { + final List responses = new ArrayList<>(shardSearchRequests.size()); + for (int i = 0; i < results.length(); i++) { + final Object o = results.get(i); + if (o instanceof CanMatchShardResponse) { + responses.add(new CanMatchResponse.ResponseOrFailure((CanMatchShardResponse) o)); + } else { + assert o instanceof Exception; + responses.add(new CanMatchResponse.ResponseOrFailure((Exception) o)); + } } + final CanMatchResponse response = new CanMatchResponse(responses); + listener.onResponse(response); + } + }; + for (int i = 0; i < shardSearchRequests.size(); i++) { + final ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); + final int finalI = i; + try { + sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { + @Override + public void onResponse(CanMatchShardResponse response) { + results.set(finalI, response); + maybeFinish.run(); + } - private void maybeFinish() { - if (counter.incrementAndGet() == shardSearchRequests.size()) { - List responses = new ArrayList<>(shardSearchRequests.size()); - List failures = new ArrayList<>(shardSearchRequests.size()); - for (int i = 0; i < results.length(); i++) { - Object o = results.get(i); - if (o instanceof CanMatchShardResponse) { - responses.add(i, (CanMatchShardResponse) o); - } else { - assert o instanceof Exception; - failures.add(i, (Exception) o); - } - } - CanMatchResponse response = new CanMatchResponse(responses, failures); - listener.onResponse(response); + @Override + public void onFailure(Exception e) { + results.set(finalI, e); + maybeFinish.run(); } - } - }); + }); + } catch (Exception e) { + results.set(finalI, e); + maybeFinish.run(); + } } } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 7af74beec0763..38134249ff963 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -1224,21 +1224,18 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { - List shardSearchRequests = request.createShardSearchRequests(); - List responses = new ArrayList<>(shardSearchRequests.size()); - List failures = new ArrayList<>(shardSearchRequests.size()); - for (int i = 0; i < shardSearchRequests.size(); i++) { - ShardSearchRequest shardSearchRequest = shardSearchRequests.get(i); + final List shardSearchRequests = request.createShardSearchRequests(); + final List responses = new ArrayList<>(shardSearchRequests.size()); + for (ShardSearchRequest shardSearchRequest : shardSearchRequests) { CanMatchShardResponse canMatchShardResponse; try { canMatchShardResponse = canMatch(shardSearchRequest); - responses.add(i, canMatchShardResponse); + responses.add(new CanMatchResponse.ResponseOrFailure(canMatchShardResponse)); } catch (Exception e) { - failures.add(i, e); + responses.add(new CanMatchResponse.ResponseOrFailure(e)); } - } - listener.onResponse(new CanMatchResponse(responses, failures)); + listener.onResponse(new CanMatchResponse(responses)); } /** diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 51e0939f15af8..03bcee065c77f 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.search.CanMatchResponse.ResponseOrFailure; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; @@ -47,7 +48,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -93,13 +93,13 @@ public void testFilterShards() throws InterruptedException { public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, ActionListener listener) { numRequests.incrementAndGet(); - final List responses = new ArrayList<>(); + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { - responses.add(new CanMatchShardResponse(shard.shardId().id() == 0 ? shard1 : - shard2, null)); + responses.add(new ResponseOrFailure(new CanMatchShardResponse(shard.shardId().id() == 0 ? shard1 : + shard2, null))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); } }; @@ -162,16 +162,13 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques if (fullFailure && randomBoolean()) { throw new IllegalArgumentException("boom"); } - final CanMatchShardResponse[] responses = - new CanMatchShardResponse[request.getShardLevelRequests().size()]; - final Exception[] failures = new Exception[request.getShardLevelRequests().size()]; - for (int i = 0; i < request.getShardLevelRequests().size(); i++) { - CanMatchRequest.Shard shard = request.getShardLevelRequests().get(i); + final List responses = new ArrayList<>(); + for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { boolean throwException = shard.shardId().id() != 0; if (throwException) { - failures[i] = new NullPointerException(); + responses.add(new ResponseOrFailure(new NullPointerException())); } else { - responses[i] = new CanMatchShardResponse(shard1, null); + responses.add(new ResponseOrFailure(new CanMatchShardResponse(shard1, null))); } } @@ -179,7 +176,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques if (fullFailure) { listener.onFailure(new NullPointerException()); } else { - listener.onResponse(new CanMatchResponse(Arrays.asList(responses), Arrays.asList(failures))); + listener.onResponse(new CanMatchResponse(responses)); } }).start(); } @@ -239,7 +236,7 @@ public void testSortShards() throws InterruptedException { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, ActionListener listener) { - final List responses = new ArrayList<>(); + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { Long min = rarely() ? null : randomLong(); Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); @@ -253,10 +250,10 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques } } - responses.add(new CanMatchShardResponse(canMatch, minMax)); + responses.add(new ResponseOrFailure(new CanMatchShardResponse(canMatch, minMax))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); } }; @@ -321,7 +318,7 @@ public void testInvalidSortShards() throws InterruptedException { @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, ActionListener listener) { - final List responses = new ArrayList<>(); + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { final MinAndMax minMax; if (shard.shardId().id() == numShards-1) { @@ -338,10 +335,10 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques shardToSkip.add(shard.shardId()); } } - responses.add(new CanMatchShardResponse(canMatch, minMax)); + responses.add(new ResponseOrFailure(new CanMatchShardResponse(canMatch, minMax))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); } }; @@ -647,13 +644,13 @@ void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, @Override public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, ActionListener listener) { - final List responses = new ArrayList<>(); + final List responses = new ArrayList<>(); for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { requests.add(request.createShardSearchRequest(shard)); - responses.add(new CanMatchShardResponse(true, null)); + responses.add(new ResponseOrFailure(new CanMatchShardResponse(true, null))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses, Collections.emptyList()))).start(); + new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); } }; From a65bd63a0a89e40235e4b319f3453326ee525deb Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 17:46:25 +0200 Subject: [PATCH 11/28] reset prefiltter settings --- .../elasticsearch/action/search/TransportSearchAction.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a244bf5ee9e88..a11e0ad22010a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -744,11 +744,11 @@ static boolean shouldPreFilterSearchShards(ClusterState clusterState, && (hasReadOnlyIndices(indices, clusterState) || hasPrimaryFieldSort(source))) { preFilterShardSize = 1; } else if (preFilterShardSize == null) { - preFilterShardSize = 1; + preFilterShardSize = SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE; } return searchRequest.searchType() == QUERY_THEN_FETCH // we can't do this for DFS it needs to fan out to all shards all the time - /*&& (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source))*/ - && preFilterShardSize <= numShards; + && (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source)) + && preFilterShardSize < numShards; } private static boolean hasReadOnlyIndices(String[] indices, ClusterState clusterState) { From 41b61bd77bd54e86de971b49fe54dbe13b8a601e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 6 Oct 2021 18:44:44 +0200 Subject: [PATCH 12/28] fix serialzation --- .../org/elasticsearch/action/search/CanMatchResponse.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java index 8eba36b96339f..7ece612d0fc50 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java @@ -33,7 +33,7 @@ public CanMatchResponse(List responses) { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeCollection(responses, StreamOutput::writeOptionalWriteable); + out.writeList(responses); } public List getResponses() { @@ -77,8 +77,9 @@ public ResponseOrFailure(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(response != null); - if (response != null) { + final boolean hasResponse = response != null; + out.writeBoolean(hasResponse); + if (hasResponse) { response.writeTo(out); } else { out.writeException(exception); From dd93c939b353b6ca0e56f82fb4232183ee4151d5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 7 Oct 2021 11:30:39 +0200 Subject: [PATCH 13/28] thread pool --- .../search/CanMatchPreFilterSearchPhase.java | 77 +++++++++++-------- .../action/search/SearchTransportService.java | 3 +- .../action/search/TransportSearchAction.java | 4 +- .../elasticsearch/threadpool/ThreadPool.java | 4 + .../CanMatchPreFilterSearchPhaseTests.java | 27 +++++-- 5 files changed, 76 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index e4e5c9d523ed1..253c6865a5036 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -15,6 +15,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.core.Nullable; import org.elasticsearch.index.query.CoordinatorRewriteContext; @@ -30,6 +31,7 @@ import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.MinAndMax; import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import java.io.IOException; @@ -109,23 +111,29 @@ public CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService search } } + private static boolean assertSearchCoordinationThread() { + assert Thread.currentThread().getName().contains(ThreadPool.Names.SEARCH_COORDINATION) : + "not called from the right thread " + Thread.currentThread().getName(); + return true; + } + @Override public void run() throws IOException { - if (shardsIts.size() > 0) { - checkNoMissingShards(); - Version version = request.minCompatibleShardNode(); - if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { - if (checkMinimumVersion(shardsIts) == false) { - throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", - request.minCompatibleShardNode()); - } + assert assertSearchCoordinationThread(); + checkNoMissingShards(); + Version version = request.minCompatibleShardNode(); + if (version != null && Version.CURRENT.minimumCompatibilityVersion().equals(version) == false) { + if (checkMinimumVersion(shardsIts) == false) { + throw new VersionMismatchException("One of the shards is incompatible with the required minimum version [{}]", + request.minCompatibleShardNode()); } - - runCoordinationPhase(); } + + runCoordinationPhase(); } private void runCoordinationPhase() { + assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, @@ -163,6 +171,7 @@ private void runCoordinationPhase() { } private void checkNoMissingShards() { + assert assertSearchCoordinationThread(); assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; if (request.allowPartialSearchResults() == false) { final StringBuilder missingShards = new StringBuilder(); @@ -203,7 +212,7 @@ private Map> groupByNode(GroupShardsIte return requests; } - class Round implements Runnable { + class Round extends AbstractRunnable { private final GroupShardsIterator shards; private final CountDown countDown; private final AtomicReferenceArray responses; @@ -215,19 +224,13 @@ class Round implements Runnable { } public void start() { - try { - run(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); - } - onPhaseFailure(CanMatchPreFilterSearchPhase.this, "", e); - } + executor.execute(this); } @Override - public void run() { + protected void doRun() { + assert assertSearchCoordinationThread(); final Map> requests = groupByNode(shards); for (Map.Entry> entry : requests.entrySet()) { @@ -312,6 +315,14 @@ private void finishPhase() { new Round(new GroupShardsIterator<>(remainingShards)).start(); } } + + @Override + public void onFailure(Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure("round", e); + } } private static class SendingTarget { @@ -360,7 +371,7 @@ private void finishHim() { if (logger.isDebugEnabled()) { logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); } - onPhaseFailure(this, "", e); + onPhaseFailure("finish", e); } } @@ -405,19 +416,25 @@ public void start() { return; } - try { - run(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + executor.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(new ParameterizedMessage("Failed to execute [{}] while running [{}] phase", request, getName()), e); + } + onPhaseFailure("start", e); } - onPhaseFailure(this, "", e); - } + + @Override + protected void doRun() throws IOException { + CanMatchPreFilterSearchPhase.this.run(); + } + }); } - public final void onPhaseFailure(SearchPhase phase, String msg, Exception cause) { - listener.onFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); + public final void onPhaseFailure(String msg, Exception cause) { + listener.onFailure(new SearchPhaseExecutionException(getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } public final Transport.Connection getConnection(SendingTarget sendingTarget) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index d5a4de1c634d9..10c3e3602aa3b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -424,7 +424,8 @@ public static void registerRequestHandler(TransportService transportService, Sea searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, true, CanMatchShardResponse::new); - transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SAME, CanMatchRequest::new, + + transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SEARCH_COORDINATION, CanMatchRequest::new, (request, channel, task) -> { searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a11e0ad22010a..70ed7f0b46360 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -792,8 +792,8 @@ private SearchPhase searchAsyncAction( SearchResponse.Clusters clusters) { if (preFilter) { return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup, - aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators, - timeProvider, task, (iter) -> { + aliasFilter, concreteIndexBoosts, threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, listener, + shardIterators, timeProvider, task, (iter) -> { SearchPhase action = searchAsyncAction( task, searchRequest, diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index d259283f57885..98ab4f514af5a 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -62,6 +62,7 @@ public static class Names { public static final String ANALYZE = "analyze"; public static final String WRITE = "write"; public static final String SEARCH = "search"; + public static final String SEARCH_COORDINATION = "search_coordination"; public static final String AUTO_COMPLETE = "auto_complete"; public static final String SEARCH_THROTTLED = "search_throttled"; public static final String MANAGEMENT = "management"; @@ -114,6 +115,7 @@ public static ThreadPoolType fromType(String type) { entry(Names.ANALYZE, ThreadPoolType.FIXED), entry(Names.WRITE, ThreadPoolType.FIXED), entry(Names.SEARCH, ThreadPoolType.FIXED), + entry(Names.SEARCH_COORDINATION, ThreadPoolType.FIXED), entry(Names.MANAGEMENT, ThreadPoolType.SCALING), entry(Names.FLUSH, ThreadPoolType.SCALING), entry(Names.REFRESH, ThreadPoolType.SCALING), @@ -184,6 +186,8 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, allocatedProcessors, 1000, false)); builders.put(Names.ANALYZE, new FixedExecutorBuilder(settings, Names.ANALYZE, 1, 16, false)); builders.put(Names.SEARCH, new FixedExecutorBuilder(settings, Names.SEARCH, searchThreadPoolSize(allocatedProcessors), 1000, true)); + builders.put(Names.SEARCH_COORDINATION, new FixedExecutorBuilder(settings, Names.SEARCH_COORDINATION, + halfAllocatedProcessorsMaxFive(allocatedProcessors), 1000, true)); builders.put( Names.AUTO_COMPLETE, new FixedExecutorBuilder(settings, Names.AUTO_COMPLETE, Math.max(allocatedProcessors / 4, 1), 100, true) diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 03bcee065c77f..1abd66fdd657b 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.DateFieldMapper; @@ -44,6 +43,8 @@ import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import java.io.IOException; @@ -73,6 +74,20 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase { private final CoordinatorRewriteContextProvider EMPTY_CONTEXT_PROVIDER = new StaticCoordinatorRewriteContextProviderBuilder().build(); + private TestThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getTestName()); + } + + @Override + public void tearDown() throws Exception { + terminate(threadPool); + super.tearDown(); + } + public void testFilterShards() throws InterruptedException { final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, System.nanoTime(), @@ -115,7 +130,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + Collections.emptyMap(), threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, null, shardsIter, timeProvider,null, (iter) -> new SearchPhase("test") { @Override @@ -195,7 +210,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + Collections.emptyMap(), threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, null, shardsIter, timeProvider,null, (iter) -> new SearchPhase("test") { @Override @@ -270,7 +285,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + Collections.emptyMap(), threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, null, shardsIter, timeProvider, null, (iter) -> new SearchPhase("test") { @Override @@ -355,7 +370,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques searchTransportService, (clusterAlias, node) -> lookup.get(node), Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)), - Collections.emptyMap(), EsExecutors.DIRECT_EXECUTOR_SERVICE, + Collections.emptyMap(), threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, null, shardsIter, timeProvider, null, (iter) -> new SearchPhase("test") { @Override @@ -664,7 +679,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques (clusterAlias, node) -> lookup.get(node), aliasFilters, Collections.emptyMap(), - EsExecutors.DIRECT_EXECUTOR_SERVICE, + threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION), searchRequest, null, shardsIter, From 36bb4a1f218803293a5c85daec5dc151f784c630 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 11 Oct 2021 16:05:20 +0200 Subject: [PATCH 14/28] javadoc --- .../search/CanMatchPreFilterSearchPhase.java | 71 +++++++++++-------- .../action/search/TransportSearchAction.java | 3 +- .../search/CanMatchShardResponse.java | 3 + 3 files changed, 46 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 253c6865a5036..9eb77cbc72fe2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -52,7 +52,18 @@ import static org.elasticsearch.core.Types.forciblyCast; -public class CanMatchPreFilterSearchPhase extends SearchPhase { +/** + * This search phase can be used as an initial search phase to pre-filter search shards based on query rewriting. + * The queries are rewritten against the shards and based on the rewrite result shards might be able to be excluded + * from the search. The extra round trip to the search shards is very cheap and is not subject to rejections + * which allows to fan out to more shards at the same time without running into rejections even if we are hitting a + * large portion of the clusters indices. + * This phase can also be used to pre-sort shards based on min/max values in each shard of the provided primary sort. + * When the query primary sort is perform on a field, this phase extracts the min/max value in each shard and + * sort them according to the provided order. This can be useful for instance to ensure that shards that contain recent + * data are executed first when sorting by descending timestamp. + */ +final class CanMatchPreFilterSearchPhase extends SearchPhase { private final Logger logger; private final SearchRequest request; @@ -73,7 +84,7 @@ public class CanMatchPreFilterSearchPhase extends SearchPhase { private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; - public CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService, + CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, Executor executor, SearchRequest request, @@ -129,10 +140,12 @@ public void run() throws IOException { } } - runCoordinationPhase(); + runCoordinatorRewritePhase(); } - private void runCoordinationPhase() { + // tries to pre-filter shards based on information that's available to the coordinator + // without having to reach out to the actual shards + private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { @@ -140,9 +153,7 @@ private void runCoordinationPhase() { Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); List shardSearchRequests = canMatchRequest.createShardSearchRequests(); - for (int i = 0; i < shardSearchRequests.size(); i++) { - ShardSearchRequest request = shardSearchRequests.get(i); - + for (ShardSearchRequest request : shardSearchRequests) { boolean canMatch = true; CoordinatorRewriteContext coordinatorRewriteContext = coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); @@ -158,7 +169,8 @@ private void runCoordinationPhase() { } else { CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); result.setShardIndex(request.shardRequestIndex()); - results.consumeResult(result, () -> {}); + results.consumeResult(result, () -> { + }); } } } @@ -166,7 +178,7 @@ private void runCoordinationPhase() { if (matchedShardLevelRequests.isEmpty() == false) { new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).start(); } else { - finishHim(); + finishPhase(); } } @@ -212,22 +224,26 @@ private Map> groupByNode(GroupShardsIte return requests; } + /** + * Sending can-match requests is round-based and grouped per target node. + * If there are failures during a round, there will be a follow-up round + * to retry on other available shard copies. + */ class Round extends AbstractRunnable { private final GroupShardsIterator shards; private final CountDown countDown; - private final AtomicReferenceArray responses; + private final AtomicReferenceArray failedResponses; Round(GroupShardsIterator shards) { this.shards = shards; this.countDown = new CountDown(shards.size()); - this.responses = new AtomicReferenceArray<>(shardsIts.size()); + this.failedResponses = new AtomicReferenceArray<>(shardsIts.size()); } - public void start() { + void start() { executor.execute(this); } - @Override protected void doRun() { assert assertSearchCoordinationThread(); @@ -282,7 +298,7 @@ public void onFailure(Exception e) { } private void onOperation(int idx, CanMatchShardResponse response) { - responses.set(idx, response); + failedResponses.set(idx, null); results.consumeResult(response, () -> { if (countDown.countDown()) { finishPhase(); @@ -291,7 +307,7 @@ private void onOperation(int idx, CanMatchShardResponse response) { } private void onOperationFailed(int idx, Exception e) { - responses.set(idx, e); + failedResponses.set(idx, e); results.consumeShardFailure(idx); if (countDown.countDown()) { finishPhase(); @@ -302,14 +318,13 @@ private void finishPhase() { List remainingShards = new ArrayList<>(); for (SearchShardIterator ssi : shards) { int shardIndex = shardItIndexMap.get(ssi); - Object resp = responses.get(shardIndex); - if (resp instanceof Exception) { - // do something meaningful + Exception failedResponse = failedResponses.get(shardIndex); + if (failedResponse != null) { remainingShards.add(ssi); } } if (remainingShards.isEmpty()) { - finishHim(); + CanMatchPreFilterSearchPhase.this.finishPhase(); } else { // trigger another round new Round(new GroupShardsIterator<>(remainingShards)).start(); @@ -359,12 +374,11 @@ private CanMatchRequest createCanMatchRequest(Map.Entry ssi.getOriginalIndices() != null); assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); - final CanMatchRequest canMatchRequest = new CanMatchRequest(first.getOriginalIndices(), request, + return new CanMatchRequest(first.getOriginalIndices(), request, shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); - return canMatchRequest; } - private void finishHim() { + private void finishPhase() { try { phaseFactory.apply(getIterator(results, shardsIts)).start(); } catch (Exception e) { @@ -377,13 +391,12 @@ private void finishHim() { private static final float DEFAULT_INDEX_BOOST = 1.0f; - public final CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { + public CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - CanMatchRequest.Shard shardRequest = new CanMatchRequest.Shard(shardIt.shardId(), + return new CanMatchRequest.Shard(shardIt.shardId(), shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); - return shardRequest; } private boolean checkMinimumVersion(GroupShardsIterator shardsIts) { @@ -391,7 +404,7 @@ private boolean checkMinimumVersion(GroupShardsIterator sha if (it.getTargetNodeIds().isEmpty() == false) { boolean isCompatible = it.getTargetNodeIds().stream().anyMatch(nodeId -> { Transport.Connection conn = getConnection(new SendingTarget(it.getClusterAlias(), nodeId)); - return conn == null ? true : conn.getVersion().onOrAfter(request.minCompatibleShardNode()); + return conn == null || conn.getVersion().onOrAfter(request.minCompatibleShardNode()); }); if (isCompatible == false) { return false; @@ -433,11 +446,11 @@ protected void doRun() throws IOException { } - public final void onPhaseFailure(String msg, Exception cause) { + public void onPhaseFailure(String msg, Exception cause) { listener.onFailure(new SearchPhaseExecutionException(getName(), msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } - public final Transport.Connection getConnection(SendingTarget sendingTarget) { + public Transport.Connection getConnection(SendingTarget sendingTarget) { Transport.Connection conn = nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId); Version minVersion = request.minCompatibleShardNode(); if (minVersion != null && conn != null && conn.getVersion().before(minVersion)) { @@ -571,7 +584,7 @@ private static Comparator shardComparator(GroupShardsIterator shardsIts.get(index)); + return comparator.thenComparing(shardsIts::get); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 70ed7f0b46360..9ad223fb249f6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -688,8 +688,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea BiFunction connectionLookup = buildConnectionLookup(searchRequest.getLocalClusterAlias(), nodes::get, remoteConnections, searchTransportService::getConnection); final Executor asyncSearchExecutor = asyncSearchExecutor(concreteLocalIndices); - final boolean preFilterSearchShards = - shouldPreFilterSearchShards(clusterState, searchRequest, concreteLocalIndices, + final boolean preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, concreteLocalIndices, localShardIterators.size() + remoteShardIterators.size()); searchAsyncActionProvider.asyncSearchAction( task, searchRequest, asyncSearchExecutor, shardIterators, timeProvider, connectionLookup, clusterState, diff --git a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java index e1dd84635445f..0a9c98e0c9013 100644 --- a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java +++ b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java @@ -15,6 +15,9 @@ import java.io.IOException; +/** + * Shard-level response for can-match requests + */ public final class CanMatchShardResponse extends SearchPhaseResult { private final boolean canMatch; private final MinAndMax estimatedMinAndMax; From da5707f14935601ddcabb9b0d55b4545961534c5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 11 Oct 2021 16:23:02 +0200 Subject: [PATCH 15/28] merge conflict --- .../action/search/CanMatchPreFilterSearchPhaseTests.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 4f32de01172aa..61d4a1dfb1dfd 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -24,8 +24,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.query.AbstractQueryBuilder; @@ -47,6 +45,7 @@ import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; +import org.elasticsearch.xcontent.NamedXContentRegistry; import java.io.IOException; import java.util.ArrayList; From ea79ae25d2c095aaffdb37a2aa253e9dc7e0479b Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 12 Oct 2021 10:34:10 +0200 Subject: [PATCH 16/28] docs --- docs/reference/modules/threadpool.asciidoc | 5 +++++ .../search/CanMatchPreFilterSearchPhase.java | 16 +++++++++------- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index 558f0f2f00583..c7242941cc671 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -21,6 +21,11 @@ There are several thread pools, but the important ones include: For count/search/suggest/get operations on `search_throttled indices`. Thread pool type is `fixed` with a size of `1`, and queue_size of `100`. +`search_coordination`:: + For lightweight search-related coordination operations. Thread pool type is + `fixed` with a size of a max of `min(5, (`<>`) / 2)`, and queue_size of `1000`. + `get`:: For get operations. Thread pool type is `fixed` with a size of <>, diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 9eb77cbc72fe2..176e0a5193f81 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -176,7 +176,7 @@ private void runCoordinatorRewritePhase() { } if (matchedShardLevelRequests.isEmpty() == false) { - new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).start(); + new Round(new GroupShardsIterator<>(matchedShardLevelRequests)).run(); } else { finishPhase(); } @@ -240,10 +240,6 @@ class Round extends AbstractRunnable { this.failedResponses = new AtomicReferenceArray<>(shardsIts.size()); } - void start() { - executor.execute(this); - } - @Override protected void doRun() { assert assertSearchCoordinationThread(); @@ -326,8 +322,13 @@ private void finishPhase() { if (remainingShards.isEmpty()) { CanMatchPreFilterSearchPhase.this.finishPhase(); } else { - // trigger another round - new Round(new GroupShardsIterator<>(remainingShards)).start(); + // trigger another round, forcing execution + executor.execute(new Round(new GroupShardsIterator<>(remainingShards)) { + @Override + public boolean isForceExecution() { + return true; + } + }); } } @@ -429,6 +430,7 @@ public void start() { return; } + // Note that the search is failed when this task is rejected by the executor executor.execute(new AbstractRunnable() { @Override public void onFailure(Exception e) { From 5a81c07a4e7995f48e42a79274758123d72a63c0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 11:03:43 +0200 Subject: [PATCH 17/28] remove dead code --- .../org/elasticsearch/transport/TransportActionProxy.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index 36ee37300fe86..6ccf879755316 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -63,13 +63,6 @@ private boolean assertConsistentTaskType(Task proxyTask, TransportRequest wrappe } } - interface ResendingContext { - String getAction(); - TransportRequest wrappedRequest(); - DiscoveryNode targetNode(); - TransportRequestOptions options(); - } - private static class ProxyResponseHandler implements TransportResponseHandler { private final Writeable.Reader reader; From e26160fed448cb13e0a5e50fa573b0c1e74f1d73 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 11:12:08 +0200 Subject: [PATCH 18/28] avoid cast --- .../action/search/SearchTransportService.java | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 10c3e3602aa3b..73d00789ff29b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -131,7 +131,6 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final ActionListener listener) { - // TODO: use minNodeVersion here to check (i.e. min{connection.getVersion(), targetNode.getVersion()}) if (connection.getVersion().onOrAfter(Version.V_8_0_0) && connection.getNode().getVersion().onOrAfter(Version.V_8_0_0)) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, @@ -139,19 +138,13 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest } else { // BWC layer: translate into shard-level requests final List shardSearchRequests = request.createShardSearchRequests(); - final AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); + final AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); final CountDown counter = new CountDown(shardSearchRequests.size()); final Runnable maybeFinish = () -> { if (counter.countDown()) { final List responses = new ArrayList<>(shardSearchRequests.size()); for (int i = 0; i < results.length(); i++) { - final Object o = results.get(i); - if (o instanceof CanMatchShardResponse) { - responses.add(new CanMatchResponse.ResponseOrFailure((CanMatchShardResponse) o)); - } else { - assert o instanceof Exception; - responses.add(new CanMatchResponse.ResponseOrFailure((Exception) o)); - } + responses.set(i, results.get(i)); } final CanMatchResponse response = new CanMatchResponse(responses); listener.onResponse(response); @@ -164,18 +157,18 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { @Override public void onResponse(CanMatchShardResponse response) { - results.set(finalI, response); + results.set(finalI, new CanMatchResponse.ResponseOrFailure(response)); maybeFinish.run(); } @Override public void onFailure(Exception e) { - results.set(finalI, e); + results.set(finalI, new CanMatchResponse.ResponseOrFailure(e)); maybeFinish.run(); } }); } catch (Exception e) { - results.set(finalI, e); + results.set(finalI, new CanMatchResponse.ResponseOrFailure(e)); maybeFinish.run(); } } From 02c719362388e7e31bd2c2aa78c059ff12c7ade1 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 11:24:18 +0200 Subject: [PATCH 19/28] simpler and rename --- .../search/CanMatchPreFilterSearchPhase.java | 47 +++++++++---------- 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 176e0a5193f81..c633fa9bf062e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -150,29 +150,26 @@ private void runCoordinatorRewritePhase() { final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, - Collections.singletonList(buildShardLevelRequest(searchShardIterator)), getNumShards(), - timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); - List shardSearchRequests = canMatchRequest.createShardSearchRequests(); - for (ShardSearchRequest request : shardSearchRequests) { - boolean canMatch = true; - CoordinatorRewriteContext coordinatorRewriteContext = - coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); - if (coordinatorRewriteContext != null) { - try { - canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); - } catch (Exception e) { - // treat as if shard is still a potential match - } - } - if (canMatch) { - matchedShardLevelRequests.add(searchShardIterator); - } else { - CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); - result.setShardIndex(request.shardRequestIndex()); - results.consumeResult(result, () -> { - }); + Collections.emptyList(), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); + final ShardSearchRequest request = canMatchRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); + boolean canMatch = true; + CoordinatorRewriteContext coordinatorRewriteContext = + coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); + if (coordinatorRewriteContext != null) { + try { + canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); + } catch (Exception e) { + // treat as if shard is still a potential match } } + if (canMatch) { + matchedShardLevelRequests.add(searchShardIterator); + } else { + CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); + result.setShardIndex(request.shardRequestIndex()); + results.consumeResult(result, () -> { + }); + } } if (matchedShardLevelRequests.isEmpty() == false) { @@ -297,7 +294,7 @@ private void onOperation(int idx, CanMatchShardResponse response) { failedResponses.set(idx, null); results.consumeResult(response, () -> { if (countDown.countDown()) { - finishPhase(); + finishRound(); } }); } @@ -306,11 +303,11 @@ private void onOperationFailed(int idx, Exception e) { failedResponses.set(idx, e); results.consumeShardFailure(idx); if (countDown.countDown()) { - finishPhase(); + finishRound(); } } - private void finishPhase() { + private void finishRound() { List remainingShards = new ArrayList<>(); for (SearchShardIterator ssi : shards) { int shardIndex = shardItIndexMap.get(ssi); @@ -320,7 +317,7 @@ private void finishPhase() { } } if (remainingShards.isEmpty()) { - CanMatchPreFilterSearchPhase.this.finishPhase(); + finishPhase(); } else { // trigger another round, forcing execution executor.execute(new Round(new GroupShardsIterator<>(remainingShards)) { From 9670258fa580f4c94764ed79c79ed907e8416599 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 12:32:02 +0200 Subject: [PATCH 20/28] checkstyle --- .../org/elasticsearch/action/search/SearchTransportService.java | 1 - server/src/main/java/org/elasticsearch/search/SearchService.java | 1 - 2 files changed, 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 9978bd7428482..031960f5a848d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -54,7 +54,6 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 82f31ea69ccb8..31b320d27ee4a 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -17,7 +17,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; -import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.CanMatchResponse; import org.elasticsearch.action.search.CanMatchRequest; import org.elasticsearch.action.search.SearchRequest; From 803068b3c7431d55433a66898943912475b7947f Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 13:24:14 +0200 Subject: [PATCH 21/28] cosmetics --- .../org/elasticsearch/action/search/SearchTransportService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 031960f5a848d..7176c6429e4e8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -144,7 +144,7 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest if (counter.countDown()) { final CanMatchResponse.ResponseOrFailure[] responses = new CanMatchResponse.ResponseOrFailure[shardSearchRequests.size()]; - for (int i = 0; i < results.length(); i++) { + for (int i = 0; i < responses.length; i++) { responses[i] = results.get(i); } final CanMatchResponse response = new CanMatchResponse(Arrays.asList(responses)); From a3457efe010b8dc1fae7e4c283c782d6138a2863 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 14:53:13 +0200 Subject: [PATCH 22/28] original indices --- .../search/CanMatchPreFilterSearchPhase.java | 8 ++--- .../action/search/CanMatchRequest.java | 34 +++++-------------- 2 files changed, 11 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index c633fa9bf062e..9a2ac44c4209e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -149,7 +149,7 @@ private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { - final CanMatchRequest canMatchRequest = new CanMatchRequest(searchShardIterator.getOriginalIndices(), request, + final CanMatchRequest canMatchRequest = new CanMatchRequest(request, Collections.emptyList(), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); final ShardSearchRequest request = canMatchRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); boolean canMatch = true; @@ -369,10 +369,8 @@ private CanMatchRequest createCanMatchRequest(Map.Entry shardLevelRequests = entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); assert entry.getValue().stream().allMatch(Objects::nonNull); - assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices() != null); - assert entry.getValue().stream().allMatch(ssi -> ssi.getOriginalIndices().equals(first.getOriginalIndices())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); - return new CanMatchRequest(first.getOriginalIndices(), request, + return new CanMatchRequest(request, shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); } @@ -393,7 +391,7 @@ public CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - return new CanMatchRequest.Shard(shardIt.shardId(), + return new CanMatchRequest.Shard(shardIt.getOriginalIndices(), shardIt.shardId(), shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 8645b074c4372..9a6735f9e0b06 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -8,9 +8,7 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -35,9 +33,8 @@ /** * Node-level request used during can-match phase */ -public class CanMatchRequest extends TransportRequest implements IndicesRequest { +public class CanMatchRequest extends TransportRequest { - private final OriginalIndices originalIndices; private final SearchSourceBuilder source; private final List shards; private final SearchType searchType; @@ -50,6 +47,7 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest private final String clusterAlias; public static class Shard implements Writeable { + private final OriginalIndices originalIndices; private final ShardId shardId; private final int shardRequestIndex; private final AliasFilter aliasFilter; @@ -57,12 +55,14 @@ public static class Shard implements Writeable { private final ShardSearchContextId readerId; private final TimeValue keepAlive; - public Shard(ShardId shardId, + public Shard(OriginalIndices originalIndices, + ShardId shardId, int shardRequestIndex, AliasFilter aliasFilter, float indexBoost, ShardSearchContextId readerId, TimeValue keepAlive) { + this.originalIndices = originalIndices; this.shardId = shardId; this.shardRequestIndex = shardRequestIndex; this.aliasFilter = aliasFilter; @@ -73,6 +73,7 @@ public Shard(ShardId shardId, } public Shard(StreamInput in) throws IOException { + originalIndices = OriginalIndices.readOriginalIndices(in); shardId = new ShardId(in); shardRequestIndex = in.readVInt(); aliasFilter = new AliasFilter(in); @@ -84,6 +85,7 @@ public Shard(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { + OriginalIndices.writeOriginalIndices(originalIndices, out); shardId.writeTo(out); out.writeVInt(shardRequestIndex); aliasFilter.writeTo(out); @@ -102,14 +104,12 @@ public ShardId shardId() { } public CanMatchRequest( - OriginalIndices originalIndices, SearchRequest searchRequest, List shards, int numberOfShards, long nowInMillis, @Nullable String clusterAlias ) { - this.originalIndices = originalIndices; this.source = searchRequest.source(); this.shards = new ArrayList<>(shards); this.searchType = searchRequest.searchType(); @@ -127,7 +127,6 @@ public CanMatchRequest( public CanMatchRequest(StreamInput in) throws IOException { super(in); source = in.readOptionalWriteable(SearchSourceBuilder::new); - originalIndices = OriginalIndices.readOriginalIndices(in); searchType = SearchType.fromId(in.readByte()); scroll = in.readOptionalWriteable(Scroll::new); requestCache = in.readOptionalBoolean(); @@ -142,7 +141,6 @@ public CanMatchRequest(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeOptionalWriteable(source); - OriginalIndices.writeOriginalIndices(originalIndices, out); out.writeByte(searchType.id()); out.writeOptionalWriteable(scroll); out.writeOptionalBoolean(requestCache); @@ -153,22 +151,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeList(shards); } - @Override - public String[] indices() { - if (originalIndices == null) { - return null; - } - return originalIndices.indices(); - } - - @Override - public IndicesOptions indicesOptions() { - if (originalIndices == null) { - return null; - } - return originalIndices.indicesOptions(); - } - public List getShardLevelRequests() { return shards; } @@ -179,7 +161,7 @@ public List createShardSearchRequests() { public ShardSearchRequest createShardSearchRequest(Shard r) { ShardSearchRequest shardSearchRequest = new ShardSearchRequest( - originalIndices, r.shardId, r.shardRequestIndex, numberOfShards, searchType, + r.originalIndices, r.shardId, r.shardRequestIndex, numberOfShards, searchType, source, requestCache, r.aliasFilter, r.indexBoost, allowPartialSearchResults, scroll, nowInMillis, clusterAlias, r.readerId, r.keepAlive ); From d032fefa1f606c7ae003d2015cd93a8b90d974f4 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 14 Oct 2021 10:50:26 -0400 Subject: [PATCH 23/28] Add can match qa test --- .../upgrades/SearchStatesIT.java | 35 +++++++++++++++++-- .../org/elasticsearch/search/CCSDuelIT.java | 6 +++- 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/SearchStatesIT.java b/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/SearchStatesIT.java index 72d86df63beaf..6f9689385e3b5 100644 --- a/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/SearchStatesIT.java +++ b/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/SearchStatesIT.java @@ -198,7 +198,7 @@ static int indexDocs(RestHighLevelClient client, String index, int numDocs) thro return numDocs; } - void verifySearch(String localIndex, int localNumDocs, String remoteIndex, int remoteNumDocs) { + void verifySearch(String localIndex, int localNumDocs, String remoteIndex, int remoteNumDocs, Integer preFilterShardSize) { try (RestHighLevelClient localClient = newLocalClient()) { Request request = new Request("POST", "/_search"); final int expectedDocs; @@ -212,6 +212,12 @@ void verifySearch(String localIndex, int localNumDocs, String remoteIndex, int r if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { request.addParameter("ccs_minimize_roundtrips", Boolean.toString(randomBoolean())); } + if (preFilterShardSize == null && randomBoolean()) { + preFilterShardSize = randomIntBetween(1, 100); + } + if (preFilterShardSize != null) { + request.addParameter("pre_filter_shard_size", Integer.toString(preFilterShardSize)); + } int size = between(1, 100); request.setJsonEntity("{\"sort\": \"f\", \"size\": " + size + "}"); Response response = localClient.getLowLevelClient().performRequest(request); @@ -245,7 +251,32 @@ public void testBWCSearchStates() throws Exception { configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); int iterations = between(1, 20); for (int i = 0; i < iterations; i++) { - verifySearch(localIndex, localNumDocs, CLUSTER_ALIAS + ":" + remoteIndex, remoteNumDocs); + verifySearch(localIndex, localNumDocs, CLUSTER_ALIAS + ":" + remoteIndex, remoteNumDocs, null); + } + localClient.indices().delete(new DeleteIndexRequest(localIndex), RequestOptions.DEFAULT); + remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); + } + } + + public void testCanMatch() throws Exception { + String localIndex = "test_can_match_local_index"; + String remoteIndex = "test_can_match_remote_index"; + try (RestHighLevelClient localClient = newLocalClient(); + RestHighLevelClient remoteClient = newRemoteClient()) { + localClient.indices().create(new CreateIndexRequest(localIndex) + .settings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(5, 20))), + RequestOptions.DEFAULT); + int localNumDocs = indexDocs(localClient, localIndex, between(10, 100)); + + remoteClient.indices().create(new CreateIndexRequest(remoteIndex) + .settings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(5, 20))), + RequestOptions.DEFAULT); + int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 100)); + + configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); + int iterations = between(1, 10); + for (int i = 0; i < iterations; i++) { + verifySearch(localIndex, localNumDocs, CLUSTER_ALIAS + ":" + remoteIndex, remoteNumDocs, between(1, 10)); } localClient.indices().delete(new DeleteIndexRequest(localIndex), RequestOptions.DEFAULT); remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); diff --git a/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java b/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java index c2b9dbdb44462..3eb698d3d0c0e 100644 --- a/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java +++ b/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java @@ -724,7 +724,11 @@ private static void assumeMultiClusterSetup() { private static SearchRequest initSearchRequest() { List indices = Arrays.asList(INDEX_NAME, "my_remote_cluster:" + INDEX_NAME); Collections.shuffle(indices, random()); - return new SearchRequest(indices.toArray(new String[0])); + final SearchRequest request = new SearchRequest(indices.toArray(new String[0])); + if (randomBoolean()) { + request.setPreFilterShardSize(between(1, 20)); + } + return request; } private static void duelSearch(SearchRequest searchRequest, Consumer responseChecker) throws Exception { From 24baa8bb48d48b35a8dd8f366447e1799c2c43ca Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 17:12:09 +0200 Subject: [PATCH 24/28] implement indicesrequest --- .../action/search/CanMatchRequest.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 9a6735f9e0b06..4115e0d57be96 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -8,7 +8,9 @@ package org.elasticsearch.action.search; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -33,7 +35,7 @@ /** * Node-level request used during can-match phase */ -public class CanMatchRequest extends TransportRequest { +public class CanMatchRequest extends TransportRequest implements IndicesRequest { private final SearchSourceBuilder source; private final List shards; @@ -45,6 +47,7 @@ public class CanMatchRequest extends TransportRequest { private final long nowInMillis; @Nullable private final String clusterAlias; + private final String[] indices; public static class Shard implements Writeable { private final OriginalIndices originalIndices; @@ -122,6 +125,7 @@ public CanMatchRequest( this.numberOfShards = numberOfShards; this.nowInMillis = nowInMillis; this.clusterAlias = clusterAlias; + indices = shards.stream().map(Shard::shardId).map(ShardId::getIndexName).distinct().toArray(String[]::new); } public CanMatchRequest(StreamInput in) throws IOException { @@ -135,6 +139,7 @@ public CanMatchRequest(StreamInput in) throws IOException { nowInMillis = in.readVLong(); clusterAlias = in.readOptionalString(); shards = in.readList(Shard::new); + indices = shards.stream().map(Shard::shardId).map(ShardId::getIndexName).distinct().toArray(String[]::new); } @Override @@ -169,6 +174,16 @@ public ShardSearchRequest createShardSearchRequest(Shard r) { return shardSearchRequest; } + @Override + public String[] indices() { + return indices; + } + + @Override + public IndicesOptions indicesOptions() { + return shards.isEmpty() ? null : shards.iterator().next().originalIndices.indicesOptions(); + } + @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { return new SearchShardTask(id, type, action, getDescription(), parentTaskId, headers); From 259f049b455eb3c7d8ac5aadb92644934deadc75 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 17:29:36 +0200 Subject: [PATCH 25/28] merge originalindices --- .../elasticsearch/action/search/CanMatchRequest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index 4115e0d57be96..e6314c8d190ce 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -101,6 +102,10 @@ public int getShardRequestIndex() { return shardRequestIndex; } + public OriginalIndices getOriginalIndices() { + return originalIndices; + } + public ShardId shardId() { return shardId; } @@ -125,7 +130,8 @@ public CanMatchRequest( this.numberOfShards = numberOfShards; this.nowInMillis = nowInMillis; this.clusterAlias = clusterAlias; - indices = shards.stream().map(Shard::shardId).map(ShardId::getIndexName).distinct().toArray(String[]::new); + indices = shards.stream().map(Shard::getOriginalIndices).map(OriginalIndices::indices).flatMap(Arrays::stream).distinct() + .toArray(String[]::new); } public CanMatchRequest(StreamInput in) throws IOException { @@ -139,7 +145,8 @@ public CanMatchRequest(StreamInput in) throws IOException { nowInMillis = in.readVLong(); clusterAlias = in.readOptionalString(); shards = in.readList(Shard::new); - indices = shards.stream().map(Shard::shardId).map(ShardId::getIndexName).distinct().toArray(String[]::new); + indices = shards.stream().map(Shard::getOriginalIndices).map(OriginalIndices::indices).flatMap(Arrays::stream).distinct() + .toArray(String[]::new); } @Override From 16f45edac1292830d600e183b5e172c46c81f3e2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Oct 2021 17:41:07 +0200 Subject: [PATCH 26/28] better --- .../search/CanMatchPreFilterSearchPhase.java | 8 +++--- .../action/search/CanMatchRequest.java | 27 +++++++++++-------- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 9a2ac44c4209e..380496ea66155 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -149,7 +149,7 @@ private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { - final CanMatchRequest canMatchRequest = new CanMatchRequest(request, + final CanMatchRequest canMatchRequest = new CanMatchRequest(request, searchShardIterator.getOriginalIndices().indicesOptions(), Collections.emptyList(), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); final ShardSearchRequest request = canMatchRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); boolean canMatch = true; @@ -369,8 +369,10 @@ private CanMatchRequest createCanMatchRequest(Map.Entry shardLevelRequests = entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); assert entry.getValue().stream().allMatch(Objects::nonNull); + assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getOriginalIndices().indicesOptions(), + first.getOriginalIndices().indicesOptions())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); - return new CanMatchRequest(request, + return new CanMatchRequest(request, first.getOriginalIndices().indicesOptions(), shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); } @@ -391,7 +393,7 @@ public CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); - return new CanMatchRequest.Shard(shardIt.getOriginalIndices(), shardIt.shardId(), + return new CanMatchRequest.Shard(shardIt.getOriginalIndices().indices(), shardIt.shardId(), shardItIndexMap.get(shardIt), filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java index e6314c8d190ce..c7eb37550ee2f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java @@ -49,9 +49,10 @@ public class CanMatchRequest extends TransportRequest implements IndicesRequest @Nullable private final String clusterAlias; private final String[] indices; + private final IndicesOptions indicesOptions; public static class Shard implements Writeable { - private final OriginalIndices originalIndices; + private final String[] indices; private final ShardId shardId; private final int shardRequestIndex; private final AliasFilter aliasFilter; @@ -59,14 +60,14 @@ public static class Shard implements Writeable { private final ShardSearchContextId readerId; private final TimeValue keepAlive; - public Shard(OriginalIndices originalIndices, + public Shard(String[] indices, ShardId shardId, int shardRequestIndex, AliasFilter aliasFilter, float indexBoost, ShardSearchContextId readerId, TimeValue keepAlive) { - this.originalIndices = originalIndices; + this.indices = indices; this.shardId = shardId; this.shardRequestIndex = shardRequestIndex; this.aliasFilter = aliasFilter; @@ -77,7 +78,7 @@ public Shard(OriginalIndices originalIndices, } public Shard(StreamInput in) throws IOException { - originalIndices = OriginalIndices.readOriginalIndices(in); + indices = in.readStringArray(); shardId = new ShardId(in); shardRequestIndex = in.readVInt(); aliasFilter = new AliasFilter(in); @@ -89,7 +90,7 @@ public Shard(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - OriginalIndices.writeOriginalIndices(originalIndices, out); + out.writeStringArray(indices); shardId.writeTo(out); out.writeVInt(shardRequestIndex); aliasFilter.writeTo(out); @@ -102,8 +103,8 @@ public int getShardRequestIndex() { return shardRequestIndex; } - public OriginalIndices getOriginalIndices() { - return originalIndices; + public String[] getOriginalIndices() { + return indices; } public ShardId shardId() { @@ -113,12 +114,14 @@ public ShardId shardId() { public CanMatchRequest( SearchRequest searchRequest, + IndicesOptions indicesOptions, List shards, int numberOfShards, long nowInMillis, @Nullable String clusterAlias ) { this.source = searchRequest.source(); + this.indicesOptions = indicesOptions; this.shards = new ArrayList<>(shards); this.searchType = searchRequest.searchType(); this.requestCache = searchRequest.requestCache(); @@ -130,13 +133,14 @@ public CanMatchRequest( this.numberOfShards = numberOfShards; this.nowInMillis = nowInMillis; this.clusterAlias = clusterAlias; - indices = shards.stream().map(Shard::getOriginalIndices).map(OriginalIndices::indices).flatMap(Arrays::stream).distinct() + indices = shards.stream().map(Shard::getOriginalIndices).flatMap(Arrays::stream).distinct() .toArray(String[]::new); } public CanMatchRequest(StreamInput in) throws IOException { super(in); source = in.readOptionalWriteable(SearchSourceBuilder::new); + indicesOptions = IndicesOptions.readIndicesOptions(in); searchType = SearchType.fromId(in.readByte()); scroll = in.readOptionalWriteable(Scroll::new); requestCache = in.readOptionalBoolean(); @@ -145,7 +149,7 @@ public CanMatchRequest(StreamInput in) throws IOException { nowInMillis = in.readVLong(); clusterAlias = in.readOptionalString(); shards = in.readList(Shard::new); - indices = shards.stream().map(Shard::getOriginalIndices).map(OriginalIndices::indices).flatMap(Arrays::stream).distinct() + indices = shards.stream().map(Shard::getOriginalIndices).flatMap(Arrays::stream).distinct() .toArray(String[]::new); } @@ -153,6 +157,7 @@ public CanMatchRequest(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeOptionalWriteable(source); + indicesOptions.writeIndicesOptions(out); out.writeByte(searchType.id()); out.writeOptionalWriteable(scroll); out.writeOptionalBoolean(requestCache); @@ -173,7 +178,7 @@ public List createShardSearchRequests() { public ShardSearchRequest createShardSearchRequest(Shard r) { ShardSearchRequest shardSearchRequest = new ShardSearchRequest( - r.originalIndices, r.shardId, r.shardRequestIndex, numberOfShards, searchType, + new OriginalIndices(r.indices, indicesOptions), r.shardId, r.shardRequestIndex, numberOfShards, searchType, source, requestCache, r.aliasFilter, r.indexBoost, allowPartialSearchResults, scroll, nowInMillis, clusterAlias, r.readerId, r.keepAlive ); @@ -188,7 +193,7 @@ public String[] indices() { @Override public IndicesOptions indicesOptions() { - return shards.isEmpty() ? null : shards.iterator().next().originalIndices.indicesOptions(); + return indicesOptions; } @Override From c0e35ee4f508888d7d9601a593f53cb72d293e59 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 15 Oct 2021 11:26:08 +0200 Subject: [PATCH 27/28] radnomize use of can-match phase --- .../action/search/TransportSearchAction.java | 12 +++++-- .../common/settings/ClusterSettings.java | 1 + .../search/TransportSearchActionTests.java | 33 ++++++++++--------- .../elasticsearch/test/ESIntegTestCase.java | 6 +++- 4 files changed, 32 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 0798a4e532a91..8fdb49c7dc7a9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -105,6 +105,9 @@ public class TransportSearchAction extends HandledTransportAction SHARD_COUNT_LIMIT_SETTING = Setting.longSetting( "action.search.shard_count.limit", Long.MAX_VALUE, 1L, Property.Dynamic, Property.NodeScope); + public static final Setting DEFAULT_PRE_FILTER_SHARD_SIZE = Setting.intSetting( + "action.search.pre_filter_shard_size.default", SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE, 1, Property.NodeScope); + private final ThreadPool threadPool; private final ClusterService clusterService; private final SearchTransportService searchTransportService; @@ -115,6 +118,7 @@ public class TransportSearchAction extends HandledTransportAction buildPerIndexOriginalIndices(ClusterState clusterState, @@ -733,7 +738,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea nodes::get, remoteConnections, searchTransportService::getConnection); final Executor asyncSearchExecutor = asyncSearchExecutor(concreteLocalIndices); final boolean preFilterSearchShards = shouldPreFilterSearchShards(clusterState, searchRequest, concreteLocalIndices, - localShardIterators.size() + remoteShardIterators.size()); + localShardIterators.size() + remoteShardIterators.size(), defaultPreFilterShardSize); searchAsyncActionProvider.asyncSearchAction( task, searchRequest, asyncSearchExecutor, shardIterators, timeProvider, connectionLookup, clusterState, Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener, @@ -780,14 +785,15 @@ static BiFunction buildConnectionLookup(St static boolean shouldPreFilterSearchShards(ClusterState clusterState, SearchRequest searchRequest, String[] indices, - int numShards) { + int numShards, + int defaultPreFilterShardSize) { SearchSourceBuilder source = searchRequest.source(); Integer preFilterShardSize = searchRequest.getPreFilterShardSize(); if (preFilterShardSize == null && (hasReadOnlyIndices(indices, clusterState) || hasPrimaryFieldSort(source))) { preFilterShardSize = 1; } else if (preFilterShardSize == null) { - preFilterShardSize = SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE; + preFilterShardSize = defaultPreFilterShardSize; } return searchRequest.searchType() == QUERY_THEN_FETCH // we can't do this for DFS it needs to fan out to all shards all the time && (SearchService.canRewriteToMatchNone(source) || hasPrimaryFieldSort(source)) diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index f03b49cda379d..2b2c64a1b430e 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -293,6 +293,7 @@ public void apply(Settings value, Settings current, Settings previous) { SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS, TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, + TransportSearchAction.DEFAULT_PRE_FILTER_SHARD_SIZE, RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE, SniffConnectionStrategy.REMOTE_CONNECTIONS_PER_CLUSTER, RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, diff --git a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java index c76ef34298f9b..b367deae020fb 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -91,6 +91,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.elasticsearch.action.search.SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE; import static org.elasticsearch.test.InternalAggregationTestCase.emptyReduceContextBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.awaitLatch; import static org.hamcrest.CoreMatchers.containsString; @@ -883,34 +884,34 @@ public void testShouldPreFilterSearchShards() { { SearchRequest searchRequest = new SearchRequest(); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 128))); + indices, randomIntBetween(2, 128), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(129, 10000))); + indices, randomIntBetween(129, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().query(QueryBuilders.rangeQuery("timestamp"))); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 128))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(129, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE + 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp"))); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 127))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE - 1), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(127, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE - 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp"))) .scroll("5m"); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 128))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(129, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE + 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } } @@ -933,35 +934,35 @@ public void testShouldPreFilterSearchShardsWithReadOnly() { { SearchRequest searchRequest = new SearchRequest(); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 127))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE - 1), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(127, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE - 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().query(QueryBuilders.rangeQuery("timestamp"))); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 127))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE - 1), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(127, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE - 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().query(QueryBuilders.rangeQuery("timestamp"))); searchRequest.scroll("5s"); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 127))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE - 1), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertTrue(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(127, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE - 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } { SearchRequest searchRequest = new SearchRequest() .source(new SearchSourceBuilder().query(QueryBuilders.rangeQuery("timestamp"))); searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(2, 127))); + indices, randomIntBetween(2, DEFAULT_PRE_FILTER_SHARD_SIZE - 1), DEFAULT_PRE_FILTER_SHARD_SIZE)); assertFalse(TransportSearchAction.shouldPreFilterSearchShards(clusterState, searchRequest, - indices, randomIntBetween(127, 10000))); + indices, randomIntBetween(DEFAULT_PRE_FILTER_SHARD_SIZE - 1, 10000), DEFAULT_PRE_FILTER_SHARD_SIZE)); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index a053ea6f5c2cb..d011624439111 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -43,7 +43,9 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DestructiveOperations; import org.elasticsearch.action.support.IndicesOptions; @@ -1754,7 +1756,9 @@ protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { // randomly enable low-level search cancellation to make sure it does not alter results .put(SearchService.LOW_LEVEL_CANCELLATION_SETTING.getKey(), randomBoolean()) .putList(DISCOVERY_SEED_HOSTS_SETTING.getKey()) // empty list disables a port scan for other nodes - .putList(DISCOVERY_SEED_PROVIDERS_SETTING.getKey(), "file"); + .putList(DISCOVERY_SEED_PROVIDERS_SETTING.getKey(), "file") + .put(TransportSearchAction.DEFAULT_PRE_FILTER_SHARD_SIZE.getKey(), randomFrom(1, 2, + SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE)); return builder.build(); } From cc8380804125eb064d8e4672e8ddde3f7a6903f9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Oct 2021 08:59:12 +0200 Subject: [PATCH 28/28] rename --- ...hRequest.java => CanMatchNodeRequest.java} | 6 +-- ...esponse.java => CanMatchNodeResponse.java} | 6 +-- .../search/CanMatchPreFilterSearchPhase.java | 35 ++++++++-------- .../action/search/SearchTransportService.java | 25 +++++------ .../elasticsearch/search/SearchService.java | 14 +++---- .../elasticsearch/threadpool/ThreadPool.java | 3 +- .../CanMatchPreFilterSearchPhaseTests.java | 42 +++++++++---------- 7 files changed, 66 insertions(+), 65 deletions(-) rename server/src/main/java/org/elasticsearch/action/search/{CanMatchRequest.java => CanMatchNodeRequest.java} (97%) rename server/src/main/java/org/elasticsearch/action/search/{CanMatchResponse.java => CanMatchNodeResponse.java} (92%) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeRequest.java similarity index 97% rename from server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java rename to server/src/main/java/org/elasticsearch/action/search/CanMatchNodeRequest.java index 25d04b86573af..1edd28798b819 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeRequest.java @@ -36,7 +36,7 @@ /** * Node-level request used during can-match phase */ -public class CanMatchRequest extends TransportRequest implements IndicesRequest { +public class CanMatchNodeRequest extends TransportRequest implements IndicesRequest { private final SearchSourceBuilder source; private final List shards; @@ -118,7 +118,7 @@ public ShardId shardId() { } } - public CanMatchRequest( + public CanMatchNodeRequest( SearchRequest searchRequest, IndicesOptions indicesOptions, List shards, @@ -144,7 +144,7 @@ public CanMatchRequest( .toArray(String[]::new); } - public CanMatchRequest(StreamInput in) throws IOException { + public CanMatchNodeRequest(StreamInput in) throws IOException { super(in); source = in.readOptionalWriteable(SearchSourceBuilder::new); indicesOptions = IndicesOptions.readIndicesOptions(in); diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java similarity index 92% rename from server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java rename to server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java index 7ece612d0fc50..05aaaa56583ed 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchNodeResponse.java @@ -18,16 +18,16 @@ import java.io.IOException; import java.util.List; -public class CanMatchResponse extends TransportResponse { +public class CanMatchNodeResponse extends TransportResponse { private final List responses; - public CanMatchResponse(StreamInput in) throws IOException { + public CanMatchNodeResponse(StreamInput in) throws IOException { super(in); responses = in.readList(ResponseOrFailure::new); } - public CanMatchResponse(List responses) { + public CanMatchNodeResponse(List responses) { this.responses = responses; } diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 6dab8b0198da5..2f26026b5d70d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -149,9 +149,10 @@ private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { - final CanMatchRequest canMatchRequest = new CanMatchRequest(request, searchShardIterator.getOriginalIndices().indicesOptions(), + final CanMatchNodeRequest canMatchNodeRequest = + new CanMatchNodeRequest(request, searchShardIterator.getOriginalIndices().indicesOptions(), Collections.emptyList(), getNumShards(), timeProvider.getAbsoluteStartMillis(), searchShardIterator.getClusterAlias()); - final ShardSearchRequest request = canMatchRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); + final ShardSearchRequest request = canMatchNodeRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); boolean canMatch = true; CoordinatorRewriteContext coordinatorRewriteContext = coordinatorRewriteContextProvider.getCoordinatorRewriteContext(request.shardId().getIndex()); @@ -243,25 +244,25 @@ protected void doRun() { final Map> requests = groupByNode(shards); for (Map.Entry> entry : requests.entrySet()) { - CanMatchRequest canMatchRequest = createCanMatchRequest(entry); - List shardLevelRequests = canMatchRequest.getShardLevelRequests(); + CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry); + List shardLevelRequests = canMatchNodeRequest.getShardLevelRequests(); if (entry.getKey().nodeId == null) { // no target node: just mark the requests as failed - for (CanMatchRequest.Shard shard : shardLevelRequests) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { onOperationFailed(shard.getShardRequestIndex(), null); } continue; } try { - searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchRequest, + searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchNodeRequest, task, new ActionListener<>() { @Override - public void onResponse(CanMatchResponse canMatchResponse) { - assert canMatchResponse.getResponses().size() == canMatchRequest.getShardLevelRequests().size(); - for (int i = 0; i < canMatchResponse.getResponses().size(); i++) { - CanMatchResponse.ResponseOrFailure response = canMatchResponse.getResponses().get(i); + public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { + assert canMatchNodeResponse.getResponses().size() == canMatchNodeRequest.getShardLevelRequests().size(); + for (int i = 0; i < canMatchNodeResponse.getResponses().size(); i++) { + CanMatchNodeResponse.ResponseOrFailure response = canMatchNodeResponse.getResponses().get(i); if (response.getResponse() != null) { CanMatchShardResponse shardResponse = response.getResponse(); shardResponse.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); @@ -276,14 +277,14 @@ public void onResponse(CanMatchResponse canMatchResponse) { @Override public void onFailure(Exception e) { - for (CanMatchRequest.Shard shard : shardLevelRequests) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { onOperationFailed(shard.getShardRequestIndex(), e); } } } ); } catch (Exception e) { - for (CanMatchRequest.Shard shard : shardLevelRequests) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { onOperationFailed(shard.getShardRequestIndex(), e); } } @@ -364,15 +365,15 @@ public int hashCode() { } } - private CanMatchRequest createCanMatchRequest(Map.Entry> entry) { + private CanMatchNodeRequest createCanMatchRequest(Map.Entry> entry) { final SearchShardIterator first = entry.getValue().get(0); - final List shardLevelRequests = + final List shardLevelRequests = entry.getValue().stream().map(this::buildShardLevelRequest).collect(Collectors.toCollection(ArrayList::new)); assert entry.getValue().stream().allMatch(Objects::nonNull); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getOriginalIndices().indicesOptions(), first.getOriginalIndices().indicesOptions())); assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); - return new CanMatchRequest(request, first.getOriginalIndices().indicesOptions(), + return new CanMatchNodeRequest(request, first.getOriginalIndices().indicesOptions(), shardLevelRequests, getNumShards(), timeProvider.getAbsoluteStartMillis(), first.getClusterAlias()); } @@ -389,12 +390,12 @@ private void finishPhase() { private static final float DEFAULT_INDEX_BOOST = 1.0f; - public CanMatchRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { + public CanMatchNodeRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); int shardRequestIndex = shardItIndexMap.get(shardIt); - return new CanMatchRequest.Shard(shardIt.getOriginalIndices().indices(), shardIt.shardId(), + return new CanMatchNodeRequest.Shard(shardIt.getOriginalIndices().indices(), shardIt.shardId(), shardRequestIndex, filter, indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive(), ShardSearchRequest.computeWaitForCheckpoint(request.getWaitForCheckpoints(), shardIt.shardId(), shardRequestIndex)); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 7176c6429e4e8..08572627fb789 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -129,25 +129,26 @@ public void sendCanMatch(Transport.Connection connection, final ShardSearchReque TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchShardResponse::new)); } - public void sendCanMatch(Transport.Connection connection, final CanMatchRequest request, SearchTask task, final - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, final CanMatchNodeRequest request, SearchTask task, final + ActionListener listener) { if (connection.getVersion().onOrAfter(Version.V_8_0_0) && connection.getNode().getVersion().onOrAfter(Version.V_8_0_0)) { transportService.sendChildRequest(connection, QUERY_CAN_MATCH_NODE_NAME, request, task, - TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchResponse::new)); + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(listener, CanMatchNodeResponse::new)); } else { // BWC layer: translate into shard-level requests final List shardSearchRequests = request.createShardSearchRequests(); - final AtomicReferenceArray results = new AtomicReferenceArray<>(shardSearchRequests.size()); + final AtomicReferenceArray results = + new AtomicReferenceArray<>(shardSearchRequests.size()); final CountDown counter = new CountDown(shardSearchRequests.size()); final Runnable maybeFinish = () -> { if (counter.countDown()) { - final CanMatchResponse.ResponseOrFailure[] responses = - new CanMatchResponse.ResponseOrFailure[shardSearchRequests.size()]; + final CanMatchNodeResponse.ResponseOrFailure[] responses = + new CanMatchNodeResponse.ResponseOrFailure[shardSearchRequests.size()]; for (int i = 0; i < responses.length; i++) { responses[i] = results.get(i); } - final CanMatchResponse response = new CanMatchResponse(Arrays.asList(responses)); + final CanMatchNodeResponse response = new CanMatchNodeResponse(Arrays.asList(responses)); listener.onResponse(response); } }; @@ -158,18 +159,18 @@ public void sendCanMatch(Transport.Connection connection, final CanMatchRequest sendCanMatch(connection, shardSearchRequest, task, new ActionListener<>() { @Override public void onResponse(CanMatchShardResponse response) { - results.set(finalI, new CanMatchResponse.ResponseOrFailure(response)); + results.set(finalI, new CanMatchNodeResponse.ResponseOrFailure(response)); maybeFinish.run(); } @Override public void onFailure(Exception e) { - results.set(finalI, new CanMatchResponse.ResponseOrFailure(e)); + results.set(finalI, new CanMatchNodeResponse.ResponseOrFailure(e)); maybeFinish.run(); } }); } catch (Exception e) { - results.set(finalI, new CanMatchResponse.ResponseOrFailure(e)); + results.set(finalI, new CanMatchNodeResponse.ResponseOrFailure(e)); maybeFinish.run(); } } @@ -419,11 +420,11 @@ public static void registerRequestHandler(TransportService transportService, Sea }); TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NAME, true, CanMatchShardResponse::new); - transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SEARCH_COORDINATION, CanMatchRequest::new, + transportService.registerRequestHandler(QUERY_CAN_MATCH_NODE_NAME, ThreadPool.Names.SEARCH_COORDINATION, CanMatchNodeRequest::new, (request, channel, task) -> { searchService.canMatch(request, new ChannelActionListener<>(channel, QUERY_CAN_MATCH_NAME, request)); }); - TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NODE_NAME, true, CanMatchResponse::new); + TransportActionProxy.registerProxyAction(transportService, QUERY_CAN_MATCH_NODE_NAME, true, CanMatchNodeResponse::new); } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index cbdf59cc60e05..50c8ab4159bfe 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -18,8 +18,8 @@ import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; -import org.elasticsearch.action.search.CanMatchRequest; -import org.elasticsearch.action.search.CanMatchResponse; +import org.elasticsearch.action.search.CanMatchNodeRequest; +import org.elasticsearch.action.search.CanMatchNodeResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; @@ -1323,19 +1323,19 @@ public void canMatch(ShardSearchRequest request, ActionListener listener) { + public void canMatch(CanMatchNodeRequest request, ActionListener listener) { final List shardSearchRequests = request.createShardSearchRequests(); - final List responses = new ArrayList<>(shardSearchRequests.size()); + final List responses = new ArrayList<>(shardSearchRequests.size()); for (ShardSearchRequest shardSearchRequest : shardSearchRequests) { CanMatchShardResponse canMatchShardResponse; try { canMatchShardResponse = canMatch(shardSearchRequest); - responses.add(new CanMatchResponse.ResponseOrFailure(canMatchShardResponse)); + responses.add(new CanMatchNodeResponse.ResponseOrFailure(canMatchShardResponse)); } catch (Exception e) { - responses.add(new CanMatchResponse.ResponseOrFailure(e)); + responses.add(new CanMatchNodeResponse.ResponseOrFailure(e)); } } - listener.onResponse(new CanMatchResponse(responses)); + listener.onResponse(new CanMatchNodeResponse(responses)); } /** diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index ece1125761b03..b3cc56a839dd0 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -186,8 +186,7 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, allocatedProcessors, 1000, false)); builders.put(Names.ANALYZE, new FixedExecutorBuilder(settings, Names.ANALYZE, 1, 16, false)); builders.put(Names.SEARCH, new FixedExecutorBuilder(settings, Names.SEARCH, searchThreadPoolSize(allocatedProcessors), 1000, true)); - builders.put(Names.SEARCH_COORDINATION, new FixedExecutorBuilder(settings, Names.SEARCH_COORDINATION, - halfAllocatedProcessorsMaxFive(allocatedProcessors), 1000, true)); + builders.put(Names.SEARCH_COORDINATION, new FixedExecutorBuilder(settings, Names.SEARCH_COORDINATION, halfProcMaxAt5, 1000, true)); builders.put( Names.AUTO_COMPLETE, new FixedExecutorBuilder(settings, Names.AUTO_COMPLETE, Math.max(allocatedProcessors / 4, 1), 100, true) diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 61d4a1dfb1dfd..d21e09707dbc5 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.search.CanMatchResponse.ResponseOrFailure; +import org.elasticsearch.action.search.CanMatchNodeResponse.ResponseOrFailure; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; @@ -105,16 +105,16 @@ public void testFilterShards() throws InterruptedException { final AtomicInteger numRequests = new AtomicInteger(); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, CanMatchNodeRequest request, SearchTask task, + ActionListener listener) { numRequests.incrementAndGet(); final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { responses.add(new ResponseOrFailure(new CanMatchShardResponse(shard.shardId().id() == 0 ? shard1 : shard2, null))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses))).start(); } }; @@ -172,13 +172,13 @@ public void testFilterWithFailure() throws InterruptedException { final boolean fullFailure = randomBoolean(); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, CanMatchNodeRequest request, SearchTask task, + ActionListener listener) { if (fullFailure && randomBoolean()) { throw new IllegalArgumentException("boom"); } final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { boolean throwException = shard.shardId().id() != 0; if (throwException) { responses.add(new ResponseOrFailure(new NullPointerException())); @@ -191,7 +191,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques if (fullFailure) { listener.onFailure(new NullPointerException()); } else { - listener.onResponse(new CanMatchResponse(responses)); + listener.onResponse(new CanMatchNodeResponse(responses)); } }).start(); } @@ -249,10 +249,10 @@ public void testSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, CanMatchNodeRequest request, SearchTask task, + ActionListener listener) { final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { Long min = rarely() ? null : randomLong(); Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); @@ -268,7 +268,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques responses.add(new ResponseOrFailure(new CanMatchShardResponse(canMatch, minMax))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses))).start(); } }; @@ -331,10 +331,10 @@ public void testInvalidSortShards() throws InterruptedException { SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, CanMatchNodeRequest request, SearchTask task, + ActionListener listener) { final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { final MinAndMax minMax; if (shard.shardId().id() == numShards-1) { minMax = new MinAndMax<>(new BytesRef("bar"), new BytesRef("baz")); @@ -353,7 +353,7 @@ public void sendCanMatch(Transport.Connection connection, CanMatchRequest reques responses.add(new ResponseOrFailure(new CanMatchShardResponse(canMatch, minMax))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses))).start(); } }; @@ -657,15 +657,15 @@ void assignShardsAndExecuteCanMatchPhase(DataStream dataStream, final List requests = Collections.synchronizedList(new ArrayList<>()); SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { @Override - public void sendCanMatch(Transport.Connection connection, CanMatchRequest request, SearchTask task, - ActionListener listener) { + public void sendCanMatch(Transport.Connection connection, CanMatchNodeRequest request, SearchTask task, + ActionListener listener) { final List responses = new ArrayList<>(); - for (CanMatchRequest.Shard shard : request.getShardLevelRequests()) { + for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { requests.add(request.createShardSearchRequest(shard)); responses.add(new ResponseOrFailure(new CanMatchShardResponse(true, null))); } - new Thread(() -> listener.onResponse(new CanMatchResponse(responses))).start(); + new Thread(() -> listener.onResponse(new CanMatchNodeResponse(responses))).start(); } };