From 11392decedceb75ae89fdb837e1d307d15c584d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 21 Jul 2021 11:47:49 +0200 Subject: [PATCH 01/31] WIP: Emulate "fields" on older versions We introduced the new "fields" option in search with version 7.10. With this change we are trying to do a best-effort attempt at emulating this new behaviour in mixed cluster or CCS scenarios where search requests using the "fields" option also target older nodes or clusters. In that case, currently we don't return anything in the "fields" section of the response. This change tried to emulate the fields behaviour by modifying the request to include the respective "_source" fields and then parsing them back into the "fields" section on return. This will not be fully equivalent to the post-7.10 "fields" functionality but at least try to include whatever we find in "_source" in earlier versions. Currently Draft only, needs more testing in CCS scenarios but I'm opening this here already to get some test coverage on the modifications so far. --- .../backwards/FieldsOptionEmulationIT.java | 103 ++++++ .../action/search/FetchSearchPhase.java | 11 + .../SearchQueryThenFetchAsyncAction.java | 34 +- .../action/search/TransportSearchAction.java | 163 ++++++++- .../search/WrappedQuerySearchResult.java | 327 ++++++++++++++++++ .../search/fetch/subphase/FieldFetcher.java | 2 +- .../search/query/QuerySearchResult.java | 14 +- 7 files changed, 636 insertions(+), 18 deletions(-) create mode 100644 qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java create mode 100644 server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java new file mode 100644 index 0000000000000..c43b0bf8d392a --- /dev/null +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -0,0 +1,103 @@ +/* + * 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.backwards; + +import org.apache.http.HttpHost; +import org.elasticsearch.Version; +import org.elasticsearch.backwards.IndexingIT.Node; +import org.elasticsearch.backwards.IndexingIT.Nodes; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.yaml.ObjectPath; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * In mixed cluster scenarios on 7.x we try to emulate the "fields" option introduced in 7.10 + * by running a request with "source" enabled for the requested patterns on older nodes and convert + * the resulting source entries back into the "fields" section. These tests check this in mixed cluster + * scenarios. + */ +public class FieldsOptionEmulationIT extends ESRestTestCase { + + private static String index = "test_field_newversion"; + private static String index_old = "test_field_oldversion"; + private static Nodes nodes; + private static List bwcNodes; + private static List newNodes; + private static String oldNodeName; + private static String newNodeName; + + @Before + public void prepareTestData() throws IOException { + nodes = IndexingIT.buildNodeAndVersions(client()); + bwcNodes = new ArrayList<>(nodes.getBWCNodes()); + newNodes = new ArrayList<>(nodes.getNewNodes()); + oldNodeName = bwcNodes.get(0).getNodeName(); + newNodeName = newNodes.get(0).getNodeName(); + createIndexOnNode(index, newNodeName); + createIndexOnNode(index_old, oldNodeName); + refreshAllIndices(); + } + + private void createIndexOnNode(String indexName, String nodeName) throws IOException { + if (indexExists(indexName) == false) { + createIndex(indexName, Settings.builder() + .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + "._name", nodeName).build()); + for (int i = 0; i < 5; i++) { + Request request = new Request("PUT", indexName + "/_doc/" + i); + request.setJsonEntity( + "{\"test\": \"test_" + randomAlphaOfLength(2) + "\"," + "\"obj\" : { \"foo\" : \"value_" + i + "\"} }" + ); + assertOK(client().performRequest(request)); + } + ensureGreen(indexName); + flush(indexName, true); + } + } + + @SuppressWarnings("unchecked") + public void testFieldOptionAdapter() throws Exception { + Request matchAllRequest = new Request("POST", + "test_field_*/_search"); + matchAllRequest.setJsonEntity("{\"_source\":false,\"fields\":[\"*\"]}"); + try ( + RestClient client = buildClient(restClientSettings(), newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new)) + ) { + Response response = client.performRequest(matchAllRequest); + ObjectPath responseObject = ObjectPath.createFromResponse(response); + System.out.println(Strings.toString(responseObject.toXContentBuilder(XContentType.JSON.xContent()))); + List> hits = responseObject.evaluate("hits.hits"); + assertEquals(10, hits.size()); + for (Map hit : hits) { + Map fieldsMap = (Map) hit.get("fields"); + assertNotNull(fieldsMap); + assertNotNull(fieldsMap.get("test")); + assertTrue(((List) fieldsMap.get("test")).get(0).toString().startsWith("test_")); + assertNotNull(fieldsMap.get("obj.foo")); + assertTrue(((List) fieldsMap.get("obj.foo")).get(0).toString().startsWith("value_")); + if (bwcNodes.get(0).getVersion().onOrAfter(Version.V_7_10_0)) { + // if all nodes are > 7.10 we should get full "fields" output even for subfields + assertTrue(((List) fieldsMap.get("test.keyword")).get(0).toString().startsWith("test_")); + } + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index f944feae10fa4..f6cb02233f0ae 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -8,10 +8,12 @@ package org.elasticsearch.action.search; import com.carrotsearch.hppc.IntArrayList; + import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.search.ScoreDoc; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.search.RescoreDocIds; @@ -162,11 +164,20 @@ private void executeFetch(final int shardIndex, final SearchShardTarget shardTar final CountedCollector counter, final ShardFetchSearchRequest fetchSearchRequest, final QuerySearchResult querySearchResult, final Transport.Connection connection) { + final FieldsOptionSourceAdapter adapter; + if (querySearchResult instanceof WrappedQuerySearchResult) { + adapter = ((WrappedQuerySearchResult) querySearchResult).getAdapter(); + } else { + adapter = null; + } context.getSearchTransport().sendExecuteFetch(connection, fetchSearchRequest, context.getTask(), new SearchActionListener(shardTarget, shardIndex) { @Override public void innerOnResponse(FetchSearchResult result) { try { + if (adapter != null) { + adapter.adaptResponse(result.hits().getHits()); + } progressListener.notifyFetchResult(shardIndex); counter.onResult(result); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 42de3794e8b7a..13fcb9b16dcd8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -10,7 +10,9 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.search.SearchPhaseResult; @@ -20,6 +22,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.Transport.Connection; import java.util.Map; import java.util.concurrent.Executor; @@ -64,11 +67,40 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); - getSearchTransport().sendExecuteQuery(getConnection(shard.getClusterAlias(), shard.getNodeId()), request, getTask(), listener); + final FieldsOptionSourceAdapter fieldsOptionAdapter; + Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); + if (connection.getVersion().before(Version.V_7_10_0)) { + fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + fieldsOptionAdapter.adaptRequest(request.source(), request::source); + } else { + fieldsOptionAdapter = null; + } + + getSearchTransport().sendExecuteQuery( + connection, + request, + getTask(), + new SearchActionListener(shard, listener.requestIndex) { + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + + @Override + protected void innerOnResponse(SearchPhaseResult response) { + if (response instanceof QuerySearchResult) { + response = new WrappedQuerySearchResult((QuerySearchResult) response, fieldsOptionAdapter); + } + listener.onResponse(response); + } + } + ); } @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 5688424cea128..f28934c2150b4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -8,6 +8,8 @@ package org.elasticsearch.action.search; +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; @@ -28,31 +30,38 @@ import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.ExecutorSelector; import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.search.fetch.subphase.FieldAndFormat; +import org.elasticsearch.search.fetch.subphase.FieldFetcher; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.lookup.SourceLookup; import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; import org.elasticsearch.tasks.Task; @@ -62,8 +71,10 @@ import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.Transport.Connection; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -79,6 +90,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongSupplier; import java.util.stream.Collectors; @@ -348,6 +360,98 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { source.collapse().getInnerHits().isEmpty(); } + public static FieldsOptionSourceAdapter createFieldsOptionAdapter(Connection connection, SearchSourceBuilder ccsSearchSource) { + Version version = connection.getVersion(); + if (version.before(Version.V_7_10_0)) { + List fetchFields = ccsSearchSource.fetchFields(); + if (fetchFields != null) { + if (fetchFields.isEmpty() == false) { + String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); + CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; + // We separate the "include_unmapped" field patters with wildcards from the rest in order to use less space in the + // lookup automaton + Map> partitions = fetchFields.stream() + .map(ff -> ff.field) + .collect(Collectors.partitioningBy((s -> Regex.isSimpleMatchPattern(s)))); + List unmappedWildcardPattern = partitions.get(true); + List unmappedConcreteFields = partitions.get(false); + if (unmappedWildcardPattern.isEmpty() == false) { + unmappedFieldsFetchAutomaton = new CharacterRunAutomaton( + Regex.simpleMatchToAutomaton(unmappedWildcardPattern.toArray(new String[unmappedWildcardPattern.size()])), + 100000 + ); + } + final FieldFetcher fieldFetcher = new FieldFetcher( + Collections.emptyMap(), + unmappedFieldsFetchAutomaton, + unmappedConcreteFields + ); + + return new FieldsOptionSourceAdapter() { + + private boolean removeSourceOnResponse = false; + + @Override + public void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { + FetchSourceContext fetchSource = source.fetchSource(); + // case 1: original request has source: true, but no includes/exclude -> do nothing on request + if (fetchSource != null && fetchSource.fetchSource()) { + if (fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { + // do nothing, we can get everything from source and can leave it when translating the response + } else { + // we probably need to error here + } + } + + // case 2: original request has source: false + if (fetchSource != null && fetchSource.fetchSource() == false) { + SearchSourceBuilder adaptedSource = source.shallowCopy(); + adaptedSource.fetchSource(new FetchSourceContext(true)); + adaptedSource.fetchSource(includes, null); + sourceConsumer.accept(adaptedSource); + removeSourceOnResponse = true; + } + } + + @Override + public void adaptResponse(SearchHit[] hits) { + for (SearchHit hit : hits) { + SourceLookup lookup = new SourceLookup(); + lookup.setSource(hit.getSourceAsMap()); + Map documentFields = Collections.emptyMap(); + try { + documentFields = fieldFetcher.fetch(lookup); + } catch (IOException e) { + // best effort fetching field, if this doesn't work continue + } + for (Map.Entry entry : documentFields.entrySet()) { + hit.setDocumentField(entry.getKey(), entry.getValue()); + } + if (removeSourceOnResponse) { + // original request didn't request source, so we remove it + hit.sourceRef(null); + } + } + } + }; + } + } + } + return NOOP_ADAPTER; + } + + interface FieldsOptionSourceAdapter { + default void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { + // noop + } + + default void adaptResponse(SearchHit[] searchHits) { + // noop + } + } + + private static final FieldsOptionSourceAdapter NOOP_ADAPTER = new FieldsOptionSourceAdapter() {}; + static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, @@ -361,12 +465,20 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); + final FieldsOptionSourceAdapter adapter = createFieldsOptionAdapter( + // TODO getting the connection here breaks a mock in TransportSearchActionTests#testCCSRemoteReduce + remoteClusterService.getConnection(clusterAlias), + searchRequest.source() + ); SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { + adapter.adaptResponse(searchResponse.getHits().getHits()); Map profileResults = searchResponse.getProfileResults(); SearchProfileShardResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileShardResults(profileResults); @@ -399,16 +511,45 @@ public void onFailure(Exception e) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), - clusterAlias, timeProvider.getAbsoluteStartMillis(), false); - ActionListener ccsListener = createCCSListener(clusterAlias, skipUnavailable, countDown, - skippedClusters, exceptions, searchResponseMerger, totalClusters, listener); + FieldsOptionSourceAdapter adapter = createFieldsOptionAdapter( + remoteClusterService.getConnection(clusterAlias), + searchRequest.source() + ); + SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( + parentTaskId, + searchRequest, + indices.indices(), + clusterAlias, + timeProvider.getAbsoluteStartMillis(), + false + ); + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + ActionListener ccsListener = createCCSListener( + clusterAlias, + skipUnavailable, + countDown, + skippedClusters, + exceptions, + searchResponseMerger, + totalClusters, + listener, + adapter + ); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, ccsListener); } if (localIndices != null) { - ActionListener ccsListener = createCCSListener(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, - false, countDown, skippedClusters, exceptions, searchResponseMerger, totalClusters, listener); + ActionListener ccsListener = createCCSListener( + RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, + false, + countDown, + skippedClusters, + exceptions, + searchResponseMerger, + totalClusters, + listener, + NOOP_ADAPTER + ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); localSearchConsumer.accept(ccsLocalSearchRequest, ccsListener); @@ -448,8 +589,10 @@ static void collectSearchShards(IndicesOptions indicesOptions, String preference boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); Client clusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); final String[] indices = entry.getValue().indices(); + ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices) .indicesOptions(indicesOptions).local(true).preference(preference).routing(routing); + clusterClient.admin().cluster().searchShards(searchShardsRequest, new CCSActionListener>( clusterAlias, skipUnavailable, responsesCountDown, skippedClusters, exceptions, listener) { @@ -470,11 +613,13 @@ Map createFinalResponse() { private static ActionListener createCCSListener(String clusterAlias, boolean skipUnavailable, CountDown countDown, AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, int totalClusters, - ActionListener originalListener) { + ActionListener originalListener, + FieldsOptionSourceAdapter adapter) { return new CCSActionListener(clusterAlias, skipUnavailable, countDown, skippedClusters, exceptions, originalListener) { @Override void innerOnResponse(SearchResponse searchResponse) { + adapter.adaptResponse(searchResponse.getHits().getHits()); searchResponseMerger.add(searchResponse); } diff --git a/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java b/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java new file mode 100644 index 0000000000000..32ef6e83c31cb --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java @@ -0,0 +1,327 @@ +/* + * 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.search.TotalHits; +import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; +import org.elasticsearch.common.io.stream.DelayableWriteable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.RescoreDocIds; +import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.fetch.FetchSearchResult; +import org.elasticsearch.search.internal.ShardSearchContextId; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.profile.ProfileShardResult; +import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.suggest.Suggest; + +import java.io.IOException; + +/** + * Wraps a {@link QuerySearchResult} together with an adapter that can be used to + * translate calls to pre 7.10 nodes when using the "fields" option and to translate + * the resulting response back + */ +final class WrappedQuerySearchResult extends QuerySearchResult { + + private final FieldsOptionSourceAdapter adapter; + private final QuerySearchResult original; + + WrappedQuerySearchResult(QuerySearchResult original, FieldsOptionSourceAdapter adapter) { + this.original = original; + this.adapter = adapter; + } + + /** + * @return the adapter to use or null if no adaption is necessary + */ + FieldsOptionSourceAdapter getAdapter() { + return this.adapter; + } + + @Override + public void remoteAddress(TransportAddress remoteAddress) { + original.remoteAddress(remoteAddress); + } + + @Override + public TransportAddress remoteAddress() { + return original.remoteAddress(); + } + + @Override + public void incRef() { + original.incRef(); + } + + @Override + public boolean tryIncRef() { + return original.tryIncRef(); + } + + @Override + public boolean decRef() { + return original.decRef(); + } + + @Override + public int hashCode() { + return original.hashCode(); + } + + @Override + public ShardSearchContextId getContextId() { + return original.getContextId(); + } + + @Override + public int getShardIndex() { + return original.getShardIndex(); + } + + @Override + public SearchShardTarget getSearchShardTarget() { + return original.getSearchShardTarget(); + } + + @Override + public void setSearchShardTarget(SearchShardTarget shardTarget) { + original.setSearchShardTarget(shardTarget); + } + + @Override + public void setShardIndex(int shardIndex) { + original.setShardIndex(shardIndex); + } + + @Override + public FetchSearchResult fetchResult() { + return original.fetchResult(); + } + + @Override + public ShardSearchRequest getShardSearchRequest() { + return original.getShardSearchRequest(); + } + + @Override + public void setShardSearchRequest(ShardSearchRequest shardSearchRequest) { + original.setShardSearchRequest(shardSearchRequest); + } + + @Override + public RescoreDocIds getRescoreDocIds() { + return original.getRescoreDocIds(); + } + + @Override + public boolean equals(Object obj) { + return original.equals(obj); + } + + @Override + public void setRescoreDocIds(RescoreDocIds rescoreDocIds) { + original.setRescoreDocIds(rescoreDocIds); + } + + @Override + public boolean isNull() { + return original.isNull(); + } + + @Override + public QuerySearchResult queryResult() { + return this; + } + + @Override + public void searchTimedOut(boolean searchTimedOut) { + original.searchTimedOut(searchTimedOut); + } + + @Override + public boolean searchTimedOut() { + return original.searchTimedOut(); + } + + @Override + public void terminatedEarly(boolean terminatedEarly) { + original.terminatedEarly(terminatedEarly); + } + + @Override + public Boolean terminatedEarly() { + return original.terminatedEarly(); + } + + @Override + public TopDocsAndMaxScore topDocs() { + return original.topDocs(); + } + + @Override + public boolean hasConsumedTopDocs() { + return original.hasConsumedTopDocs(); + } + + @Override + public TopDocsAndMaxScore consumeTopDocs() { + return original.consumeTopDocs(); + } + + @Override + public void topDocs(TopDocsAndMaxScore topDocs, DocValueFormat[] sortValueFormats) { + original.topDocs(topDocs, sortValueFormats); + } + + @Override + public DocValueFormat[] sortValueFormats() { + return original.sortValueFormats(); + } + + @Override + public boolean hasAggs() { + return original.hasAggs(); + } + + @Override + public InternalAggregations consumeAggs() { + return original.consumeAggs(); + } + + @Override + public void releaseAggs() { + original.releaseAggs(); + } + + @Override + public void aggregations(InternalAggregations aggregations) { + original.aggregations(aggregations); + } + + @Override + public DelayableWriteable aggregations() { + return original.aggregations(); + } + + @Override + public ProfileShardResult consumeProfileResult() { + return original.consumeProfileResult(); + } + + @Override + public boolean hasProfileResults() { + return original.hasProfileResults(); + } + + @Override + public void consumeAll() { + original.consumeAll(); + } + + @Override + public String toString() { + return original.toString(); + } + + @Override + public void profileResults(ProfileShardResult shardResults) { + original.profileResults(shardResults); + } + + @Override + public Suggest suggest() { + return original.suggest(); + } + + @Override + public void suggest(Suggest suggest) { + original.suggest(suggest); + } + + @Override + public int from() { + return original.from(); + } + + @Override + public QuerySearchResult from(int from) { + return original.from(from); + } + + @Override + public int size() { + return original.size(); + } + + @Override + public QuerySearchResult size(int size) { + return original.size(size); + } + + @Override + public long serviceTimeEWMA() { + return original.serviceTimeEWMA(); + } + + @Override + public QuerySearchResult serviceTimeEWMA(long serviceTimeEWMA) { + return original.serviceTimeEWMA(serviceTimeEWMA); + } + + @Override + public int nodeQueueSize() { + return original.nodeQueueSize(); + } + + @Override + public QuerySearchResult nodeQueueSize(int nodeQueueSize) { + return original.nodeQueueSize(nodeQueueSize); + } + + @Override + public boolean hasSuggestHits() { + return original.hasSuggestHits(); + } + + @Override + public boolean hasSearchContext() { + return original.hasSearchContext(); + } + + @Override + public void readFromWithId(ShardSearchContextId id, StreamInput in) throws IOException { + original.readFromWithId(id, in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + original.writeTo(out); + } + + @Override + public void writeToNoId(StreamOutput out) throws IOException { + original.writeToNoId(out); + } + + @Override + public TotalHits getTotalHits() { + return original.getTotalHits(); + } + + @Override + public float getMaxScore() { + return original.getMaxScore(); + } +} \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/FieldFetcher.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/FieldFetcher.java index 9f39a2d9a3877..b0aec110f4593 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/FieldFetcher.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/FieldFetcher.java @@ -141,7 +141,7 @@ private static FieldFetcher create(SearchExecutionContext context, private final CharacterRunAutomaton unmappedFieldsFetchAutomaton; private final List unmappedConcreteFields; - private FieldFetcher( + public FieldFetcher( Map fieldContexts, @Nullable CharacterRunAutomaton unmappedFieldsFetchAutomaton, @Nullable List unmappedConcreteFields diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index b48c61b5ba9fd..28cf624a7b3bb 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -8,12 +8,6 @@ package org.elasticsearch.search.query; -import static java.util.Collections.emptyList; -import static org.elasticsearch.common.lucene.Lucene.readTopDocs; -import static org.elasticsearch.common.lucene.Lucene.writeTopDocs; - -import java.io.IOException; - import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.TotalHits; import org.elasticsearch.Version; @@ -33,7 +27,13 @@ import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.suggest.Suggest; -public final class QuerySearchResult extends SearchPhaseResult { +import java.io.IOException; + +import static java.util.Collections.emptyList; +import static org.elasticsearch.common.lucene.Lucene.readTopDocs; +import static org.elasticsearch.common.lucene.Lucene.writeTopDocs; + +public class QuerySearchResult extends SearchPhaseResult { private int from; private int size; From 813d011454dfb2b8806317b6b88cedaab9d360cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 28 Jul 2021 17:53:00 +0200 Subject: [PATCH 02/31] Adding ccs testing --- .../upgrades/SearchStatesIT.java | 81 +++++++++++++++++++ .../backwards/FieldsOptionEmulationIT.java | 3 - 2 files changed, 81 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 8e7f3a399e3df..d1b4867ca7c7a 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 @@ -42,12 +42,15 @@ import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.action.document.RestIndexAction; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; @@ -55,8 +58,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -260,4 +265,80 @@ public void testBWCSearchStates() throws Exception { remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); } } + + // TODO move test to its own module where local and remote conditions can be controlled better + public void testFieldsOptionEmulation() throws Exception { + String localIndex = "test_bwc_fields_index"; + String remoteIndex = "test_bwc_fields_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(1, 5))), + RequestOptions.DEFAULT); + int localNumDocs = indexDocs(localClient, localIndex, between(10, 20)); + // if not fully upgraded, we should find one node with older version + Optional oldVersionNode = getNodes(remoteClient.getLowLevelClient()).stream().filter(n -> n.version.before(Version.CURRENT)).findFirst(); + if (oldVersionNode.isPresent()) { + System.out.println(oldVersionNode.get()); + } + Builder remoteIndexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5)); + if (oldVersionNode.isPresent()) { + remoteIndexSettings.put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + "._name", oldVersionNode.get().name); + } + remoteClient.indices().create(new CreateIndexRequest(remoteIndex) + .settings(remoteIndexSettings), + RequestOptions.DEFAULT); + int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20)); + int expectedHitCount = localNumDocs + remoteNumDocs; + + configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); + // TODO other test uses 20 iterations, don't understand why, should this do as well? + RestClient lowLevelClient = localClient.getLowLevelClient(); + List localNodes = getNodes(lowLevelClient); + Node currentVersionNode = localNodes.stream().filter(n -> n.version.equals(Version.CURRENT)).findFirst().get(); + lowLevelClient + .setNodes( + Collections.singletonList( + new org.elasticsearch.client.Node( + HttpHost.create(currentVersionNode.httpAddress), + null, + null, + currentVersionNode.version.toString(), + null, + null + ) + ) + ); + + for (String minimizeRoundTrips : new String[] { "true", "false" }) { + Request request = new Request("POST", "/_search"); + request.addParameter("index", localIndex + "," + CLUSTER_ALIAS + ":" + remoteIndex); + if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { + request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); + } + request.setJsonEntity("{\"_source\": false, \"fields\": [\"*\"] , \"size\": " + expectedHitCount + "}"); + Response response = lowLevelClient.performRequest(request); + try ( + XContentParser parser = JsonXContent.jsonXContent.createParser( + NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + response.getEntity().getContent() + ) + ) { + SearchResponse searchResponse = SearchResponse.fromXContent(parser); + ElasticsearchAssertions.assertNoFailures(searchResponse); + ElasticsearchAssertions.assertHitCount(searchResponse, expectedHitCount); + SearchHit[] hits = searchResponse.getHits().getHits(); + for (SearchHit hit : hits) { + assertFalse("No source in hit expected but was: " + hit.toString(), hit.hasSource()); + Map fields = hit.getFields(); + assertNotNull(fields); + assertNotNull("Field `f` not found, hit was: " + hit.toString(), fields.get("f")); + } + } + } + localClient.indices().delete(new DeleteIndexRequest(localIndex), RequestOptions.DEFAULT); + remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); + } + } } diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java index c43b0bf8d392a..abec745561f5c 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -15,9 +15,7 @@ import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; import org.junit.Before; @@ -83,7 +81,6 @@ public void testFieldOptionAdapter() throws Exception { ) { Response response = client.performRequest(matchAllRequest); ObjectPath responseObject = ObjectPath.createFromResponse(response); - System.out.println(Strings.toString(responseObject.toXContentBuilder(XContentType.JSON.xContent()))); List> hits = responseObject.evaluate("hits.hits"); assertEquals(10, hits.size()); for (Map hit : hits) { From 950f98791f7987341e21dd815a82c53490c28b4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 29 Jul 2021 20:12:22 +0200 Subject: [PATCH 03/31] Move ccs testing to own qa project --- .../build.gradle | 61 ++++ .../upgrades/CCSFieldsOptionEmulationIT.java | 263 ++++++++++++++++++ .../upgrades/SearchStatesIT.java | 83 +----- .../SearchQueryThenFetchAsyncAction.java | 2 +- .../action/search/TransportSearchAction.java | 20 +- 5 files changed, 334 insertions(+), 95 deletions(-) create mode 100644 qa/ccs-old-version-remote-cluster/build.gradle create mode 100644 qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java diff --git a/qa/ccs-old-version-remote-cluster/build.gradle b/qa/ccs-old-version-remote-cluster/build.gradle new file mode 100644 index 0000000000000..3e056f24f442b --- /dev/null +++ b/qa/ccs-old-version-remote-cluster/build.gradle @@ -0,0 +1,61 @@ +/* + * 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. + */ + +import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.internal.info.BuildParams +import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask + +apply plugin: 'elasticsearch.internal-testclusters' +apply plugin: 'elasticsearch.standalone-rest-test' +apply plugin: 'elasticsearch.bwc-test' +apply plugin: 'elasticsearch.rest-resources' + +dependencies { + testImplementation project(':client:rest-high-level') +} + +for (Version bwcVersion : BuildParams.bwcVersions.wireCompatible) { + String baseName = "v${bwcVersion}" + String bwcVersionStr = "${bwcVersion}" + + testClusters { + "${baseName}-local" { + numberOfNodes = 2 + versions = [project.version] + setting 'cluster.remote.node.attr', 'gateway' + setting 'xpack.security.enabled', 'false' + } + "${baseName}-remote" { + numberOfNodes = 2 + versions = [bwcVersionStr] + firstNode.setting 'node.attr.gateway', 'true' + lastNode.setting 'node.attr.gateway', 'true' + setting 'xpack.security.enabled', 'false' + } + } + + tasks.withType(StandaloneRestIntegTestTask).matching { it.name.startsWith("${baseName}#") }.configureEach { + useCluster testClusters."${baseName}-local" + useCluster testClusters."${baseName}-remote" + systemProperty 'tests.upgrade_from_version', bwcVersionStr.replace('-SNAPSHOT', '') + + doFirst { + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}-local".allHttpSocketURI.join(",")}") + nonInputProperties.systemProperty('tests.rest.remote_cluster', "${-> testClusters."${baseName}-remote".allHttpSocketURI.join(",")}") + } + } + + tasks.register("${baseName}#testBWCEmulation", StandaloneRestIntegTestTask) { + dependsOn "processTestResources" + mustRunAfter("precommit") + } + + tasks.register(bwcTaskName(bwcVersion)) { + dependsOn tasks.named("${baseName}#testBWCEmulation") + } +} diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java new file mode 100644 index 0000000000000..abcd84950b782 --- /dev/null +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -0,0 +1,263 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.upgrades; + +import org.apache.http.HttpHost; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.indices.CreateIndexRequest; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.document.DocumentField; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.Settings.Builder; +import org.elasticsearch.common.xcontent.DeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.rest.action.document.RestIndexAction; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.yaml.ObjectPath; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; + +/** + * This test ensure that we emulate the "fields" option when the local cluster supports it but the remote + * cluster is running an older compatible version. + */ +public class CCSFieldsOptionEmulationIT extends ESRestTestCase { + + private static final Logger LOGGER = LogManager.getLogger(CCSFieldsOptionEmulationIT.class); + private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); + private static final String CLUSTER_ALIAS = "remote_cluster"; + + // TODO refactor this and SearchStateIT so that common code moves somewhere else + static class Node { + final String id; + final String name; + final Version version; + final String transportAddress; + final String httpAddress; + final Map attributes; + + Node(String id, String name, Version version, String transportAddress, String httpAddress, Map attributes) { + this.id = id; + this.name = name; + this.version = version; + this.transportAddress = transportAddress; + this.httpAddress = httpAddress; + this.attributes = attributes; + } + + @Override + public String toString() { + return "Node{" + + "id='" + id + '\'' + + ", name='" + name + '\'' + + ", version=" + version + + ", transportAddress='" + transportAddress + '\'' + + ", httpAddress='" + httpAddress + '\'' + + ", attributes=" + attributes + + '}'; + } + } + + static List getNodes(RestClient restClient) throws IOException { + Response response = restClient.performRequest(new Request("GET", "_nodes")); + ObjectPath objectPath = ObjectPath.createFromResponse(response); + final Map nodeMap = objectPath.evaluate("nodes"); + final List nodes = new ArrayList<>(); + for (String id : nodeMap.keySet()) { + final String name = objectPath.evaluate("nodes." + id + ".name"); + final Version version = Version.fromString(objectPath.evaluate("nodes." + id + ".version")); + final String transportAddress = objectPath.evaluate("nodes." + id + ".transport.publish_address"); + final String httpAddress = objectPath.evaluate("nodes." + id + ".http.publish_address"); + final Map attributes = objectPath.evaluate("nodes." + id + ".attributes"); + nodes.add(new Node(id, name, version, transportAddress, httpAddress, attributes)); + } + return nodes; + } + + static List parseHosts(String props) { + final String address = System.getProperty(props); + assertNotNull("[" + props + "] is not configured", address); + String[] stringUrls = address.split(","); + List hosts = new ArrayList<>(stringUrls.length); + for (String stringUrl : stringUrls) { + int portSeparator = stringUrl.lastIndexOf(':'); + if (portSeparator < 0) { + throw new IllegalArgumentException("Illegal cluster url [" + stringUrl + "]"); + } + String host = stringUrl.substring(0, portSeparator); + int port = Integer.parseInt(stringUrl.substring(portSeparator + 1)); + hosts.add(new HttpHost(host, port, "http")); + } + assertThat("[" + props + "] is empty", hosts, not(empty())); + return hosts; + } + + public static void configureRemoteClusters(List remoteNodes) throws Exception { + assertThat(remoteNodes, hasSize(2)); + final String remoteClusterSettingPrefix = "cluster.remote." + CLUSTER_ALIAS + "."; + try (RestHighLevelClient localClient = newLocalClient()) { + final Settings remoteConnectionSettings; + if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0) || randomBoolean()) { + final List seeds = remoteNodes.stream() + .filter(n -> n.attributes.containsKey("gateway")) + .map(n -> n.transportAddress) + .collect(Collectors.toList()); + assertThat(seeds, hasSize(2)); + LOGGER.info("--> use sniff mode with seed [{}], remote nodes [{}]", seeds, remoteNodes); + if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0)) { + remoteConnectionSettings = Settings.builder() + .putList(remoteClusterSettingPrefix + "seeds", seeds) + .build(); + } else { + remoteConnectionSettings = Settings.builder() + .putNull(remoteClusterSettingPrefix + "proxy_address") + .put(remoteClusterSettingPrefix + "mode", "sniff") + .putList(remoteClusterSettingPrefix + "seeds", seeds) + .build(); + } + } else { + final Node proxyNode = randomFrom(remoteNodes); + LOGGER.info("--> use proxy node [{}], remote nodes [{}]", proxyNode, remoteNodes); + remoteConnectionSettings = Settings.builder() + .putNull(remoteClusterSettingPrefix + "seeds") + .put(remoteClusterSettingPrefix + "mode", "proxy") + .put(remoteClusterSettingPrefix + "proxy_address", proxyNode.transportAddress) + .build(); + } + assertTrue( + localClient.cluster() + .putSettings(new ClusterUpdateSettingsRequest().persistentSettings(remoteConnectionSettings), RequestOptions.DEFAULT) + .isAcknowledged() + ); + assertBusy(() -> { + final Response resp = localClient.getLowLevelClient().performRequest(new Request("GET", "/_remote/info")); + assertOK(resp); + final ObjectPath objectPath = ObjectPath.createFromResponse(resp); + assertNotNull(objectPath.evaluate(CLUSTER_ALIAS)); + assertTrue(objectPath.evaluate(CLUSTER_ALIAS + ".connected")); + }, 60, TimeUnit.SECONDS); + } + } + + static RestHighLevelClient newLocalClient() { + final List hosts = parseHosts("tests.rest.cluster"); + final int index = random().nextInt(hosts.size()); + LOGGER.info("Using client node {}", index); + return new RestHighLevelClient(RestClient.builder(hosts.get(index))); + } + + static RestHighLevelClient newRemoteClient() { + return new RestHighLevelClient(RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster")))); + } + + static int indexDocs(RestHighLevelClient client, String index, int numDocs) throws IOException { + for (int i = 0; i < numDocs; i++) { + Request indexDoc = new Request("PUT", index + "/type/" + i); + indexDoc.setJsonEntity("{\"f\":" + i + "}"); + indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); + client.getLowLevelClient().performRequest(indexDoc); + } + client.indices().refresh(new RefreshRequest(index), RequestOptions.DEFAULT); + return numDocs; + } + + public void testFieldsOptionEmulation() throws Exception { + String localIndex = "test_bwc_fields_index"; + String remoteIndex = "test_bwc_fields_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(1, 5))), + RequestOptions.DEFAULT); + int localNumDocs = indexDocs(localClient, localIndex, between(10, 20)); + + Builder remoteIndexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5)); + remoteClient.indices().create(new CreateIndexRequest(remoteIndex) + .settings(remoteIndexSettings), + RequestOptions.DEFAULT); + int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20)); + int expectedHitCount = localNumDocs + remoteNumDocs; + + configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); + RestClient lowLevelClient = localClient.getLowLevelClient(); + for (String minimizeRoundTrips : new String[] { "true", "false" }) { + Request request = new Request("POST", "/_search"); + request.addParameter("index", localIndex + "," + CLUSTER_ALIAS + ":" + remoteIndex); + if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { + request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); + } + request.setJsonEntity("{\"_source\": false, \"fields\": [\"*\"] , \"size\": " + expectedHitCount + "}"); + Response response = lowLevelClient.performRequest(request); + try ( + XContentParser parser = JsonXContent.jsonXContent.createParser( + NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + response.getEntity().getContent() + ) + ) { + SearchResponse searchResponse = SearchResponse.fromXContent(parser); + ElasticsearchAssertions.assertNoFailures(searchResponse); + ElasticsearchAssertions.assertHitCount(searchResponse, expectedHitCount); + SearchHit[] hits = searchResponse.getHits().getHits(); + for (SearchHit hit : hits) { + assertFalse("No source in hit expected but was: " + hit.toString(), hit.hasSource()); + Map fields = hit.getFields(); + assertNotNull(fields); + assertNotNull("Field `f` not found, hit was: " + hit.toString(), fields.get("f")); + } + } + } + localClient.indices().delete(new DeleteIndexRequest(localIndex), RequestOptions.DEFAULT); + remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); + } + } +} 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 d1b4867ca7c7a..ec99a2d3dacbc 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 @@ -42,15 +42,12 @@ import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.action.document.RestIndexAction; -import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; @@ -58,10 +55,8 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -265,80 +260,4 @@ public void testBWCSearchStates() throws Exception { remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); } } - - // TODO move test to its own module where local and remote conditions can be controlled better - public void testFieldsOptionEmulation() throws Exception { - String localIndex = "test_bwc_fields_index"; - String remoteIndex = "test_bwc_fields_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(1, 5))), - RequestOptions.DEFAULT); - int localNumDocs = indexDocs(localClient, localIndex, between(10, 20)); - // if not fully upgraded, we should find one node with older version - Optional oldVersionNode = getNodes(remoteClient.getLowLevelClient()).stream().filter(n -> n.version.before(Version.CURRENT)).findFirst(); - if (oldVersionNode.isPresent()) { - System.out.println(oldVersionNode.get()); - } - Builder remoteIndexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5)); - if (oldVersionNode.isPresent()) { - remoteIndexSettings.put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + "._name", oldVersionNode.get().name); - } - remoteClient.indices().create(new CreateIndexRequest(remoteIndex) - .settings(remoteIndexSettings), - RequestOptions.DEFAULT); - int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20)); - int expectedHitCount = localNumDocs + remoteNumDocs; - - configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); - // TODO other test uses 20 iterations, don't understand why, should this do as well? - RestClient lowLevelClient = localClient.getLowLevelClient(); - List localNodes = getNodes(lowLevelClient); - Node currentVersionNode = localNodes.stream().filter(n -> n.version.equals(Version.CURRENT)).findFirst().get(); - lowLevelClient - .setNodes( - Collections.singletonList( - new org.elasticsearch.client.Node( - HttpHost.create(currentVersionNode.httpAddress), - null, - null, - currentVersionNode.version.toString(), - null, - null - ) - ) - ); - - for (String minimizeRoundTrips : new String[] { "true", "false" }) { - Request request = new Request("POST", "/_search"); - request.addParameter("index", localIndex + "," + CLUSTER_ALIAS + ":" + remoteIndex); - if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { - request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); - } - request.setJsonEntity("{\"_source\": false, \"fields\": [\"*\"] , \"size\": " + expectedHitCount + "}"); - Response response = lowLevelClient.performRequest(request); - try ( - XContentParser parser = JsonXContent.jsonXContent.createParser( - NamedXContentRegistry.EMPTY, - DeprecationHandler.THROW_UNSUPPORTED_OPERATION, - response.getEntity().getContent() - ) - ) { - SearchResponse searchResponse = SearchResponse.fromXContent(parser); - ElasticsearchAssertions.assertNoFailures(searchResponse); - ElasticsearchAssertions.assertHitCount(searchResponse, expectedHitCount); - SearchHit[] hits = searchResponse.getHits().getHits(); - for (SearchHit hit : hits) { - assertFalse("No source in hit expected but was: " + hit.toString(), hit.hasSource()); - Map fields = hit.getFields(); - assertNotNull(fields); - assertNotNull("Field `f` not found, hit was: " + hit.toString(), fields.get("f")); - } - } - } - localClient.indices().delete(new DeleteIndexRequest(localIndex), RequestOptions.DEFAULT); - remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); - } - } -} +} \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 13fcb9b16dcd8..05ebd320ad5cf 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -94,7 +94,7 @@ public void onFailure(Exception e) { @Override protected void innerOnResponse(SearchPhaseResult response) { - if (response instanceof QuerySearchResult) { + if (response instanceof QuerySearchResult && fieldsOptionAdapter != null) { response = new WrappedQuerySearchResult((QuerySearchResult) response, fieldsOptionAdapter); } listener.onResponse(response); 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 f28934c2150b4..94be9118d3d64 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -437,20 +437,14 @@ public void adaptResponse(SearchHit[] hits) { } } } - return NOOP_ADAPTER; + return null; } interface FieldsOptionSourceAdapter { - default void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { - // noop - } - - default void adaptResponse(SearchHit[] searchHits) { - // noop - } + void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer); + void adaptResponse(SearchHit[] searchHits); } - private static final FieldsOptionSourceAdapter NOOP_ADAPTER = new FieldsOptionSourceAdapter() {}; static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, @@ -548,7 +542,7 @@ public void onFailure(Exception e) { searchResponseMerger, totalClusters, listener, - NOOP_ADAPTER + null ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); @@ -614,12 +608,14 @@ private static ActionListener createCCSListener(String clusterAl AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, int totalClusters, ActionListener originalListener, - FieldsOptionSourceAdapter adapter) { + @Nullable FieldsOptionSourceAdapter adapter) { return new CCSActionListener(clusterAlias, skipUnavailable, countDown, skippedClusters, exceptions, originalListener) { @Override void innerOnResponse(SearchResponse searchResponse) { - adapter.adaptResponse(searchResponse.getHits().getHits()); + if (adapter != null) { + adapter.adaptResponse(searchResponse.getHits().getHits()); + } searchResponseMerger.add(searchResponse); } From 83fdb97dc74a93148f0012e246c7e935cec15e88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 29 Jul 2021 20:38:40 +0200 Subject: [PATCH 04/31] Prevent potential NPE --- .../search/SearchQueryThenFetchAsyncAction.java | 7 ++----- .../action/search/TransportSearchAction.java | 12 +++++++++--- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 05ebd320ad5cf..4f29eb2abeaf4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.cluster.ClusterState; @@ -74,11 +73,9 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); final FieldsOptionSourceAdapter fieldsOptionAdapter; Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); - if (connection.getVersion().before(Version.V_7_10_0)) { - fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + if (fieldsOptionAdapter != null) { fieldsOptionAdapter.adaptRequest(request.source(), request::source); - } else { - fieldsOptionAdapter = null; } getSearchTransport().sendExecuteQuery( 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 94be9118d3d64..944d01fb3dcf7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -466,13 +466,17 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or ); SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + if (adapter != null) { + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + } Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { - adapter.adaptResponse(searchResponse.getHits().getHits()); + if (adapter != null) { + adapter.adaptResponse(searchResponse.getHits().getHits()); + } Map profileResults = searchResponse.getProfileResults(); SearchProfileShardResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileShardResults(profileResults); @@ -517,7 +521,9 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + if (adapter != null) { + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + } ActionListener ccsListener = createCCSListener( clusterAlias, skipUnavailable, From a05f7b627a2b47679cec66df774cc3eacd76a9db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 30 Jul 2021 13:00:23 +0200 Subject: [PATCH 05/31] Fix warnings and mute known failing test --- .../java/org/elasticsearch/client/RestClient.java | 5 +++-- .../upgrades/CCSFieldsOptionEmulationIT.java | 11 +++++++---- .../org/elasticsearch/upgrades/SearchStatesIT.java | 2 +- .../CrossClusterSearchUnavailableClusterIT.java | 4 ++++ .../action/search/TransportSearchActionTests.java | 3 ++- 5 files changed, 17 insertions(+), 8 deletions(-) diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index beefe4cc15d50..dbb0a17eddae4 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -28,9 +28,9 @@ import org.apache.http.HttpResponse; import org.apache.http.client.AuthCache; import org.apache.http.client.ClientProtocolException; +import org.apache.http.client.config.RequestConfig; import org.apache.http.client.entity.GzipCompressingEntity; import org.apache.http.client.entity.GzipDecompressingEntity; -import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpOptions; @@ -50,7 +50,6 @@ import org.apache.http.nio.protocol.HttpAsyncRequestProducer; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; -import javax.net.ssl.SSLHandshakeException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -82,6 +81,8 @@ import java.util.stream.Collectors; import java.util.zip.GZIPOutputStream; +import javax.net.ssl.SSLHandshakeException; + import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Collections.singletonList; diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index abcd84950b782..de397aebf076c 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -199,11 +199,13 @@ static RestHighLevelClient newRemoteClient() { return new RestHighLevelClient(RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster")))); } - static int indexDocs(RestHighLevelClient client, String index, int numDocs) throws IOException { + static int indexDocs(RestHighLevelClient client, String index, int numDocs, boolean expectWarnings) throws IOException { for (int i = 0; i < numDocs; i++) { Request indexDoc = new Request("PUT", index + "/type/" + i); indexDoc.setJsonEntity("{\"f\":" + i + "}"); - indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); + if (expectWarnings) { + indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); + } client.getLowLevelClient().performRequest(indexDoc); } client.indices().refresh(new RefreshRequest(index), RequestOptions.DEFAULT); @@ -218,13 +220,14 @@ public void testFieldsOptionEmulation() throws Exception { localClient.indices().create(new CreateIndexRequest(localIndex) .settings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))), RequestOptions.DEFAULT); - int localNumDocs = indexDocs(localClient, localIndex, between(10, 20)); + int localNumDocs = indexDocs(localClient, localIndex, between(10, 20), true); Builder remoteIndexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5)); remoteClient.indices().create(new CreateIndexRequest(remoteIndex) .settings(remoteIndexSettings), RequestOptions.DEFAULT); - int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20)); + boolean expectRemoteIndexWarnings = UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0); + int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20), expectRemoteIndexWarnings); int expectedHitCount = localNumDocs + remoteNumDocs; configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); 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 ec99a2d3dacbc..a71e0c3ef6e42 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 RestHighLevelClient newRemoteClient() { static int indexDocs(RestHighLevelClient client, String index, int numDocs) throws IOException { for (int i = 0; i < numDocs; i++) { - Request indexDoc = new Request("PUT", index + "/type/" + i); + Request indexDoc = new Request("PUT", index + "/_doc/" + i); indexDoc.setJsonEntity("{\"f\":" + i + "}"); indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); client.getLowLevelClient().performRequest(indexDoc); diff --git a/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java b/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java index 00f0fb99f6c45..a839494216182 100644 --- a/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java +++ b/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java @@ -12,6 +12,7 @@ import org.apache.http.entity.ContentType; import org.apache.http.nio.entity.NStringEntity; import org.apache.lucene.search.TotalHits; +import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; @@ -61,6 +62,8 @@ import java.util.concurrent.TimeUnit; import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase { @@ -131,6 +134,7 @@ private static MockTransportService startTransport( } } + @AwaitsFix(bugUrl = "TODO enhance mock to not fail on remoteClusterService.getConnection() calls") public void testSearchSkipUnavailable() throws IOException { try (MockTransportService remoteTransport = startTransport("node0", new CopyOnWriteArrayList<>(), Version.CURRENT, threadPool)) { DiscoveryNode remoteNode = remoteTransport.getLocalDiscoNode(); 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 1ac4bb114bb3d..fbd8c2f77ab89 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -32,10 +32,10 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.core.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.index.query.MatchAllQueryBuilder; @@ -414,6 +414,7 @@ public void testCCSRemoteReduceMergeFails() throws Exception { } } + @AwaitsFix(bugUrl = "TODO enhance mock to not fail on remoteClusterService.getConnection() calls") public void testCCSRemoteReduce() throws Exception { int numClusters = randomIntBetween(1, 10); DiscoveryNode[] nodes = new DiscoveryNode[numClusters]; From ac9a8bc0ea9f05d259d13983ac75adf53cccde29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 30 Jul 2021 15:01:53 +0200 Subject: [PATCH 06/31] Fix failing tests that disconnect clusters --- ...rossClusterSearchUnavailableClusterIT.java | 4 --- .../action/search/TransportSearchAction.java | 33 ++++++++++++------- .../search/TransportSearchActionTests.java | 1 - 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java b/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java index a839494216182..00f0fb99f6c45 100644 --- a/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java +++ b/qa/ccs-unavailable-clusters/src/test/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java @@ -12,7 +12,6 @@ import org.apache.http.entity.ContentType; import org.apache.http.nio.entity.NStringEntity; import org.apache.lucene.search.TotalHits; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; @@ -62,8 +61,6 @@ import java.util.concurrent.TimeUnit; import static org.hamcrest.CoreMatchers.containsString; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThat; public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase { @@ -134,7 +131,6 @@ private static MockTransportService startTransport( } } - @AwaitsFix(bugUrl = "TODO enhance mock to not fail on remoteClusterService.getConnection() calls") public void testSearchSkipUnavailable() throws IOException { try (MockTransportService remoteTransport = startTransport("node0", new CopyOnWriteArrayList<>(), Version.CURRENT, threadPool)) { DiscoveryNode remoteNode = remoteTransport.getLocalDiscoNode(); 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 944d01fb3dcf7..81008b73e731a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -67,6 +67,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; @@ -459,23 +460,28 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - final FieldsOptionSourceAdapter adapter = createFieldsOptionAdapter( - // TODO getting the connection here breaks a mock in TransportSearchActionTests#testCCSRemoteReduce - remoteClusterService.getConnection(clusterAlias), - searchRequest.source() - ); + FieldsOptionSourceAdapter adapter = null; + try { + adapter = createFieldsOptionAdapter( + remoteClusterService.getConnection(clusterAlias), + searchRequest.source() + ); + } catch (NoSuchRemoteClusterException ex) { + // no connection version, adapter creation not possible if cluster not connected + } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); if (adapter != null) { adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); } + final FieldsOptionSourceAdapter finalAdapter = adapter; Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { - if (adapter != null) { - adapter.adaptResponse(searchResponse.getHits().getHits()); + if (finalAdapter != null) { + finalAdapter.adaptResponse(searchResponse.getHits().getHits()); } Map profileResults = searchResponse.getProfileResults(); SearchProfileShardResults profile = profileResults == null || profileResults.isEmpty() @@ -509,10 +515,15 @@ public void onFailure(Exception e) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = createFieldsOptionAdapter( - remoteClusterService.getConnection(clusterAlias), - searchRequest.source() - ); + FieldsOptionSourceAdapter adapter = null; + try { + adapter = createFieldsOptionAdapter( + remoteClusterService.getConnection(clusterAlias), + searchRequest.source() + ); + } catch (NoSuchRemoteClusterException ex) { + // don't create fields option converter in this case + } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( parentTaskId, searchRequest, 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 fbd8c2f77ab89..54689a1a00d54 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -414,7 +414,6 @@ public void testCCSRemoteReduceMergeFails() throws Exception { } } - @AwaitsFix(bugUrl = "TODO enhance mock to not fail on remoteClusterService.getConnection() calls") public void testCCSRemoteReduce() throws Exception { int numClusters = randomIntBetween(1, 10); DiscoveryNode[] nodes = new DiscoveryNode[numClusters]; From f9ee226b404269a176c3526b2801a2f99e401161 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 30 Jul 2021 16:31:15 +0200 Subject: [PATCH 07/31] adding cases to FieldsOptionEmulationIT --- .../backwards/FieldsOptionEmulationIT.java | 47 ++++++++++++++++--- 1 file changed, 41 insertions(+), 6 deletions(-) diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java index abec745561f5c..9dd57fd22bd6a 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -72,14 +72,51 @@ private void createIndexOnNode(String indexName, String nodeName) throws IOExcep } @SuppressWarnings("unchecked") - public void testFieldOptionAdapter() throws Exception { - Request matchAllRequest = new Request("POST", + public void testFieldOptionAdapterAllFields() throws Exception { + for (String includeSource : new String[] { "true", "false" }) { + Request matchAllRequest = new Request("POST", "test_field_*/_search"); + + matchAllRequest.setJsonEntity("{\"_source\":" + includeSource + " ,\"fields\":[\"*\"]}"); + try ( + RestClient client = buildClient( + restClientSettings(), + newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new) + ) + ) { + Response response = client.performRequest(matchAllRequest); + ObjectPath responseObject = ObjectPath.createFromResponse(response); + List> hits = responseObject.evaluate("hits.hits"); + assertEquals(10, hits.size()); + for (Map hit : hits) { + Map fieldsMap = (Map) hit.get("fields"); + assertNotNull(fieldsMap); + assertNotNull(fieldsMap.get("test")); + assertTrue(((List) fieldsMap.get("test")).get(0).toString().startsWith("test_")); + assertNotNull(fieldsMap.get("obj.foo")); + assertTrue(((List) fieldsMap.get("obj.foo")).get(0).toString().startsWith("value_")); + if (bwcNodes.get(0).getVersion().onOrAfter(Version.V_7_10_0)) { + // if all nodes are > 7.10 we should get full "fields" output even for subfields + assertTrue(((List) fieldsMap.get("test.keyword")).get(0).toString().startsWith("test_")); + } + if (includeSource.equals("true")) { + assertNotNull(hit.get("_source")); + } else { + assertNull(hit.get("_source")); + } + } + } + } + } + + @SuppressWarnings("unchecked") + public void testFieldOptionAdapterFilterFields() throws Exception { + Request matchAllRequestFiltered = new Request("POST", "test_field_*/_search"); - matchAllRequest.setJsonEntity("{\"_source\":false,\"fields\":[\"*\"]}"); + matchAllRequestFiltered.setJsonEntity("{\"_source\":false,\"fields\":[\"test*\"]}"); try ( RestClient client = buildClient(restClientSettings(), newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new)) ) { - Response response = client.performRequest(matchAllRequest); + Response response = client.performRequest(matchAllRequestFiltered); ObjectPath responseObject = ObjectPath.createFromResponse(response); List> hits = responseObject.evaluate("hits.hits"); assertEquals(10, hits.size()); @@ -88,8 +125,6 @@ public void testFieldOptionAdapter() throws Exception { assertNotNull(fieldsMap); assertNotNull(fieldsMap.get("test")); assertTrue(((List) fieldsMap.get("test")).get(0).toString().startsWith("test_")); - assertNotNull(fieldsMap.get("obj.foo")); - assertTrue(((List) fieldsMap.get("obj.foo")).get(0).toString().startsWith("value_")); if (bwcNodes.get(0).getVersion().onOrAfter(Version.V_7_10_0)) { // if all nodes are > 7.10 we should get full "fields" output even for subfields assertTrue(((List) fieldsMap.get("test.keyword")).get(0).toString().startsWith("test_")); From 76312386d3770f916eaaf70d14f9d17413ccb67c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 30 Jul 2021 17:11:11 +0200 Subject: [PATCH 08/31] Use noop adapter instead of nulling instances --- .../action/search/FetchSearchPhase.java | 6 ++-- .../SearchQueryThenFetchAsyncAction.java | 6 ++-- .../action/search/TransportSearchAction.java | 35 +++++++++---------- 3 files changed, 20 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index f6cb02233f0ae..3b423c12a7c75 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -168,16 +168,14 @@ private void executeFetch(final int shardIndex, final SearchShardTarget shardTar if (querySearchResult instanceof WrappedQuerySearchResult) { adapter = ((WrappedQuerySearchResult) querySearchResult).getAdapter(); } else { - adapter = null; + adapter = TransportSearchAction.NOOP_FIELDSADAPTER; } context.getSearchTransport().sendExecuteFetch(connection, fetchSearchRequest, context.getTask(), new SearchActionListener(shardTarget, shardIndex) { @Override public void innerOnResponse(FetchSearchResult result) { try { - if (adapter != null) { - adapter.adaptResponse(result.hits().getHits()); - } + adapter.adaptResponse(result.hits().getHits()); progressListener.notifyFetchResult(shardIndex); counter.onResult(result); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4f29eb2abeaf4..fe4702bdcb1f8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -74,9 +74,7 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final FieldsOptionSourceAdapter fieldsOptionAdapter; Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); - if (fieldsOptionAdapter != null) { - fieldsOptionAdapter.adaptRequest(request.source(), request::source); - } + fieldsOptionAdapter.adaptRequest(request.source(), request::source); getSearchTransport().sendExecuteQuery( connection, @@ -91,7 +89,7 @@ public void onFailure(Exception e) { @Override protected void innerOnResponse(SearchPhaseResult response) { - if (response instanceof QuerySearchResult && fieldsOptionAdapter != null) { + if (response instanceof QuerySearchResult) { response = new WrappedQuerySearchResult((QuerySearchResult) response, fieldsOptionAdapter); } listener.onResponse(response); 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 81008b73e731a..d542b6caf6335 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -438,14 +438,19 @@ public void adaptResponse(SearchHit[] hits) { } } } - return null; + return NOOP_FIELDSADAPTER; } interface FieldsOptionSourceAdapter { - void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer); - void adaptResponse(SearchHit[] searchHits); + default void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { + // noop + }; + default void adaptResponse(SearchHit[] searchHits) { + // noop + }; } + public static final FieldsOptionSourceAdapter NOOP_FIELDSADAPTER = new FieldsOptionSourceAdapter() {}; static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, @@ -460,7 +465,7 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = null; + FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; try { adapter = createFieldsOptionAdapter( remoteClusterService.getConnection(clusterAlias), @@ -471,18 +476,14 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - if (adapter != null) { - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); - } + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); final FieldsOptionSourceAdapter finalAdapter = adapter; Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { - if (finalAdapter != null) { - finalAdapter.adaptResponse(searchResponse.getHits().getHits()); - } + finalAdapter.adaptResponse(searchResponse.getHits().getHits()); Map profileResults = searchResponse.getProfileResults(); SearchProfileShardResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileShardResults(profileResults); @@ -515,7 +516,7 @@ public void onFailure(Exception e) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = null; + FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; try { adapter = createFieldsOptionAdapter( remoteClusterService.getConnection(clusterAlias), @@ -532,9 +533,7 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - if (adapter != null) { - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); - } + adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); ActionListener ccsListener = createCCSListener( clusterAlias, skipUnavailable, @@ -559,7 +558,7 @@ public void onFailure(Exception e) { searchResponseMerger, totalClusters, listener, - null + NOOP_FIELDSADAPTER ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); @@ -625,14 +624,12 @@ private static ActionListener createCCSListener(String clusterAl AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, int totalClusters, ActionListener originalListener, - @Nullable FieldsOptionSourceAdapter adapter) { + FieldsOptionSourceAdapter adapter) { return new CCSActionListener(clusterAlias, skipUnavailable, countDown, skippedClusters, exceptions, originalListener) { @Override void innerOnResponse(SearchResponse searchResponse) { - if (adapter != null) { - adapter.adaptResponse(searchResponse.getHits().getHits()); - } + adapter.adaptResponse(searchResponse.getHits().getHits()); searchResponseMerger.add(searchResponse); } From 8859c21a2595290f3591e9b2607be980c8ec42d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 30 Jul 2021 18:36:44 +0200 Subject: [PATCH 09/31] small cleanups --- .../src/main/java/org/elasticsearch/client/RestClient.java | 5 ++--- .../test/java/org/elasticsearch/upgrades/SearchStatesIT.java | 4 ++-- .../action/search/TransportSearchActionTests.java | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index dbb0a17eddae4..beefe4cc15d50 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -28,9 +28,9 @@ import org.apache.http.HttpResponse; import org.apache.http.client.AuthCache; import org.apache.http.client.ClientProtocolException; -import org.apache.http.client.config.RequestConfig; import org.apache.http.client.entity.GzipCompressingEntity; import org.apache.http.client.entity.GzipDecompressingEntity; +import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpOptions; @@ -50,6 +50,7 @@ import org.apache.http.nio.protocol.HttpAsyncRequestProducer; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import javax.net.ssl.SSLHandshakeException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -81,8 +82,6 @@ import java.util.stream.Collectors; import java.util.zip.GZIPOutputStream; -import javax.net.ssl.SSLHandshakeException; - import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Collections.singletonList; 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 a71e0c3ef6e42..8e7f3a399e3df 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 RestHighLevelClient newRemoteClient() { static int indexDocs(RestHighLevelClient client, String index, int numDocs) throws IOException { for (int i = 0; i < numDocs; i++) { - Request indexDoc = new Request("PUT", index + "/_doc/" + i); + Request indexDoc = new Request("PUT", index + "/type/" + i); indexDoc.setJsonEntity("{\"f\":" + i + "}"); indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); client.getLowLevelClient().performRequest(indexDoc); @@ -260,4 +260,4 @@ public void testBWCSearchStates() throws Exception { remoteClient.indices().delete(new DeleteIndexRequest(remoteIndex), RequestOptions.DEFAULT); } } -} \ No newline at end of file +} 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 54689a1a00d54..1ac4bb114bb3d 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -32,10 +32,10 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.core.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.index.query.MatchAllQueryBuilder; From 74fd52e6b481934fc5c89d105c68a6555641bd95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 2 Aug 2021 17:38:13 +0200 Subject: [PATCH 10/31] Add request flag to only enable bwc behaviour explicitely --- .../upgrades/CCSFieldsOptionEmulationIT.java | 1 + .../backwards/FieldsOptionEmulationIT.java | 45 ++++++++++++++- .../SearchQueryThenFetchAsyncAction.java | 8 ++- .../action/search/SearchRequest.java | 56 +++++++++++++++++-- .../action/search/TransportSearchAction.java | 27 +++++---- .../action/search/RestMultiSearchAction.java | 2 +- .../rest/action/search/RestSearchAction.java | 7 ++- .../search/builder/SearchSourceBuilder.java | 8 +-- .../search/MultiSearchRequestTests.java | 6 +- .../action/search/SearchRequestTests.java | 3 +- .../search/RandomSearchRequestGenerator.java | 5 +- 11 files changed, 135 insertions(+), 33 deletions(-) diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index de397aebf076c..3e3804689d34d 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -237,6 +237,7 @@ public void testFieldsOptionEmulation() throws Exception { request.addParameter("index", localIndex + "," + CLUSTER_ALIAS + ":" + remoteIndex); if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); + request.addParameter("enable_fields_emulation", "true"); } request.setJsonEntity("{\"_source\": false, \"fields\": [\"*\"] , \"size\": " + expectedHitCount + "}"); Response response = lowLevelClient.performRequest(request); diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java index 9dd57fd22bd6a..e7ce1f5f5bcf4 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -40,6 +40,7 @@ public class FieldsOptionEmulationIT extends ESRestTestCase { private static List newNodes; private static String oldNodeName; private static String newNodeName; + private static Version bwcNodeVersion; @Before public void prepareTestData() throws IOException { @@ -48,6 +49,7 @@ public void prepareTestData() throws IOException { newNodes = new ArrayList<>(nodes.getNewNodes()); oldNodeName = bwcNodes.get(0).getNodeName(); newNodeName = newNodes.get(0).getNodeName(); + bwcNodeVersion = bwcNodes.get(0).getVersion(); createIndexOnNode(index, newNodeName); createIndexOnNode(index_old, oldNodeName); refreshAllIndices(); @@ -75,6 +77,7 @@ private void createIndexOnNode(String indexName, String nodeName) throws IOExcep public void testFieldOptionAdapterAllFields() throws Exception { for (String includeSource : new String[] { "true", "false" }) { Request matchAllRequest = new Request("POST", "test_field_*/_search"); + matchAllRequest.addParameter("enable_fields_emulation", "true"); matchAllRequest.setJsonEntity("{\"_source\":" + includeSource + " ,\"fields\":[\"*\"]}"); try ( @@ -94,7 +97,7 @@ public void testFieldOptionAdapterAllFields() throws Exception { assertTrue(((List) fieldsMap.get("test")).get(0).toString().startsWith("test_")); assertNotNull(fieldsMap.get("obj.foo")); assertTrue(((List) fieldsMap.get("obj.foo")).get(0).toString().startsWith("value_")); - if (bwcNodes.get(0).getVersion().onOrAfter(Version.V_7_10_0)) { + if (bwcNodeVersion.onOrAfter(Version.V_7_10_0)) { // if all nodes are > 7.10 we should get full "fields" output even for subfields assertTrue(((List) fieldsMap.get("test.keyword")).get(0).toString().startsWith("test_")); } @@ -108,10 +111,50 @@ public void testFieldOptionAdapterAllFields() throws Exception { } } + /** + * test that the fields emulation is turned off by default or if 'enable_fields_emulation' is set to false + */ + public void testFieldOptionEmulationNotEnabled() throws Exception { + for (String includeSource : new String[] { "true", "false" }) { + Request matchAllRequest = new Request("POST", "test_field_*/_search"); + // enable_fields_emulation should be "false" by default, but also randomly set it on the request + if (randomBoolean()) { + matchAllRequest.addParameter("enable_fields_emulation", "false"); + } + + matchAllRequest.setJsonEntity("{\"_source\":" + includeSource + " ,\"fields\":[\"*\"]}"); + try ( + RestClient client = buildClient( + restClientSettings(), + newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new) + ) + ) { + Response response = client.performRequest(matchAllRequest); + ObjectPath responseObject = ObjectPath.createFromResponse(response); + List> hits = responseObject.evaluate("hits.hits"); + assertEquals(10, hits.size()); + for (Map hit : hits) { + String index = (String) hit.get("_index"); + if (index.equals("test_field_oldversion") && bwcNodeVersion.before(Version.V_7_10_0)) { + assertNull(hit.get("fields")); + } else { + assertNotNull(hit.get("fields")); + } + if (includeSource.equals("true")) { + assertNotNull(hit.get("_source")); + } else { + assertNull(hit.get("_source")); + } + } + } + } + } + @SuppressWarnings("unchecked") public void testFieldOptionAdapterFilterFields() throws Exception { Request matchAllRequestFiltered = new Request("POST", "test_field_*/_search"); + matchAllRequestFiltered.addParameter("enable_fields_emulation", "true"); matchAllRequestFiltered.setJsonEntity("{\"_source\":false,\"fields\":[\"test*\"]}"); try ( RestClient client = buildClient(restClientSettings(), newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new)) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index fe4702bdcb1f8..fecd5342e5b1b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -71,9 +71,13 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final SearchShardTarget shard, final SearchActionListener listener) { ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); - final FieldsOptionSourceAdapter fieldsOptionAdapter; Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); - fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + final FieldsOptionSourceAdapter fieldsOptionAdapter; + if (getRequest().isFieldsOptionEmulationEnabled()) { + fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + } else { + fieldsOptionAdapter = TransportSearchAction.NOOP_FIELDSADAPTER; + } fieldsOptionAdapter.adaptRequest(request.source(), request::source); getSearchTransport().sendExecuteQuery( diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 723b1fabe4add..5f833a85ebd91 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -13,12 +13,12 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.search.Scroll; @@ -26,8 +26,8 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.sort.FieldSortBuilder; -import org.elasticsearch.search.sort.SortBuilder; import org.elasticsearch.search.sort.ShardDocSortField; +import org.elasticsearch.search.sort.SortBuilder; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.tasks.TaskId; @@ -91,6 +91,7 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla private String[] types = Strings.EMPTY_ARRAY; private boolean ccsMinimizeRoundtrips; + private boolean enableFieldsEmulation = false; @Nullable private final Version minCompatibleShardNode; @@ -193,6 +194,7 @@ private SearchRequest(SearchRequest searchRequest, String[] indices, String loca this.absoluteStartMillis = absoluteStartMillis; this.finalReduce = finalReduce; this.minCompatibleShardNode = searchRequest.minCompatibleShardNode; + this.enableFieldsEmulation = searchRequest.enableFieldsEmulation; } /** @@ -246,6 +248,11 @@ public SearchRequest(StreamInput in) throws IOException { } else { minCompatibleShardNode = null; } + if (in.getVersion().onOrAfter(Version.V_7_15_0)) { + this.enableFieldsEmulation = in.readBoolean(); + } else { + this.enableFieldsEmulation = false; + } } @Override @@ -286,6 +293,9 @@ public void writeTo(StreamOutput out) throws IOException { Version.writeVersion(minCompatibleShardNode, out); } } + if (out.getVersion().onOrAfter(Version.V_7_15_0)) { + out.writeBoolean(enableFieldsEmulation); + } } @Override @@ -433,6 +443,21 @@ public void setCcsMinimizeRoundtrips(boolean ccsMinimizeRoundtrips) { this.ccsMinimizeRoundtrips = ccsMinimizeRoundtrips; } + /** + * Returns whether the "fields" option will be emulated via fetching from source on pre-7.10 nodes. + * The default is false. + */ + public boolean isFieldsOptionEmulationEnabled() { + return enableFieldsEmulation; + } + + /** + * Sets whether the "fields" option will be emulated via fetching from source on pre-7.10 nodes. + */ + public void setFieldsOptionEmulationEnabled(boolean enableFieldsEmulation) { + this.enableFieldsEmulation = enableFieldsEmulation; + } + /** * The document types to execute the search against. Defaults to be executed against * all types. @@ -792,14 +817,32 @@ public boolean equals(Object o) { Objects.equals(localClusterAlias, that.localClusterAlias) && absoluteStartMillis == that.absoluteStartMillis && ccsMinimizeRoundtrips == that.ccsMinimizeRoundtrips && + enableFieldsEmulation == that.enableFieldsEmulation && Objects.equals(minCompatibleShardNode, that.minCompatibleShardNode); } @Override public int hashCode() { - return Objects.hash(searchType, Arrays.hashCode(indices), routing, preference, source, requestCache, - scroll, Arrays.hashCode(types), indicesOptions, batchedReduceSize, maxConcurrentShardRequests, preFilterShardSize, - allowPartialSearchResults, localClusterAlias, absoluteStartMillis, ccsMinimizeRoundtrips, minCompatibleShardNode); + return Objects.hash( + searchType, + Arrays.hashCode(indices), + routing, + preference, + source, + requestCache, + scroll, + Arrays.hashCode(types), + indicesOptions, + batchedReduceSize, + maxConcurrentShardRequests, + preFilterShardSize, + allowPartialSearchResults, + localClusterAlias, + absoluteStartMillis, + ccsMinimizeRoundtrips, + minCompatibleShardNode, + enableFieldsEmulation + ); } @Override @@ -820,6 +863,7 @@ public String toString() { ", localClusterAlias=" + localClusterAlias + ", getOrCreateAbsoluteStartMillis=" + absoluteStartMillis + ", ccsMinimizeRoundtrips=" + ccsMinimizeRoundtrips + + ", enableFieldsEmulation=" + enableFieldsEmulation + ", source=" + source + '}'; } } 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 d542b6caf6335..2e073c5f8bd8b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -466,13 +466,12 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; - try { - adapter = createFieldsOptionAdapter( - remoteClusterService.getConnection(clusterAlias), - searchRequest.source() - ); - } catch (NoSuchRemoteClusterException ex) { - // no connection version, adapter creation not possible if cluster not connected + if (searchRequest.isFieldsOptionEmulationEnabled()) { + try { + adapter = createFieldsOptionAdapter(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); + } catch (NoSuchRemoteClusterException ex) { + // no connection version, adapter creation not possible if cluster not connected + } } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); @@ -517,13 +516,13 @@ public void onFailure(Exception e) { boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; - try { - adapter = createFieldsOptionAdapter( - remoteClusterService.getConnection(clusterAlias), - searchRequest.source() - ); - } catch (NoSuchRemoteClusterException ex) { - // don't create fields option converter in this case + if (searchRequest.isFieldsOptionEmulationEnabled()) { + try { + + adapter = createFieldsOptionAdapter(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); + } catch (NoSuchRemoteClusterException ex) { + // don't create fields option converter in this case + } } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( parentTaskId, diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 3bdb28840c1db..3946280d3cd35 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -17,13 +17,13 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.core.Tuple; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.Tuple; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestCancellableNodeClient; diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index fd2f8b3f3c70d..084bb96fdc54b 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -16,12 +16,12 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.core.Booleans; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.core.Booleans; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; @@ -144,6 +144,11 @@ public static void parseSearchRequest(SearchRequest searchRequest, RestRequest r if (request.hasParam("pre_filter_shard_size")) { searchRequest.setPreFilterShardSize(request.paramAsInt("pre_filter_shard_size", SearchRequest.DEFAULT_PRE_FILTER_SHARD_SIZE)); } + if (request.hasParam("enable_fields_emulation")) { + searchRequest.setFieldsOptionEmulationEnabled( + request.paramAsBoolean("enable_fields_emulation", searchRequest.isFieldsOptionEmulationEnabled()) + ); + } if (request.hasParam("max_concurrent_shard_requests")) { // only set if we have the parameter since we auto adjust the max concurrency on the coordinator diff --git a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index 4f1bf3b96b3aa..1f0e37c1f95a7 100644 --- a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -10,9 +10,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; -import org.elasticsearch.core.Booleans; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -20,13 +17,16 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; -import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.Booleans; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.Rewriteable; diff --git a/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java b/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java index 876dd6440278b..530a2f305f479 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java @@ -10,18 +10,18 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.CheckedBiConsumer; -import org.elasticsearch.core.CheckedRunnable; -import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.rest.RestRequest; @@ -381,6 +381,8 @@ private static MultiSearchRequest createMultiSearchRequest() { MultiSearchRequest request = new MultiSearchRequest(); for (int j = 0; j < numSearchRequest; j++) { SearchRequest searchRequest = createSimpleSearchRequest(); + // we don't support reading this request parameter in multi-line format, so supressing it here + searchRequest.setFieldsOptionEmulationEnabled(false); if (randomBoolean()) { searchRequest.allowPartialSearchResults(true); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java index 4f450a5621708..3d0da9520deea 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java @@ -13,8 +13,8 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.ArrayUtils; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.AbstractSearchTestCase; import org.elasticsearch.search.Scroll; @@ -266,6 +266,7 @@ private SearchRequest mutate(SearchRequest searchRequest) { () -> randomFrom(SearchType.DFS_QUERY_THEN_FETCH, SearchType.QUERY_THEN_FETCH)))); mutators.add(() -> mutation.source(randomValueOtherThan(searchRequest.source(), this::createSearchSourceBuilder))); mutators.add(() -> mutation.setCcsMinimizeRoundtrips(searchRequest.isCcsMinimizeRoundtrips() == false)); + mutators.add(() -> mutation.setFieldsOptionEmulationEnabled(searchRequest.isFieldsOptionEmulationEnabled() == false)); randomFrom(mutators).run(); return mutation; } diff --git a/test/framework/src/main/java/org/elasticsearch/search/RandomSearchRequestGenerator.java b/test/framework/src/main/java/org/elasticsearch/search/RandomSearchRequestGenerator.java index d03da47ceb89d..7e7d8f821fd33 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/RandomSearchRequestGenerator.java +++ b/test/framework/src/main/java/org/elasticsearch/search/RandomSearchRequestGenerator.java @@ -13,13 +13,13 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.text.Text; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; @@ -107,6 +107,9 @@ public static SearchRequest randomSearchRequest(Supplier ra if (randomBoolean()) { searchRequest.source(randomSearchSourceBuilder.get()); } + if (randomBoolean()) { + searchRequest.setFieldsOptionEmulationEnabled(randomBoolean()); + } return searchRequest; } From 6c648ea29039e300af78ef6c9b0c86b7932614bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 3 Aug 2021 15:10:24 +0200 Subject: [PATCH 11/31] iter --- .../org/elasticsearch/client/RequestConverters.java | 9 ++++++--- .../elasticsearch/client/RequestConvertersTests.java | 11 +++++++++-- .../elasticsearch/action/search/SearchRequest.java | 8 +++++--- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index 40b75504df077..2ddda555763ab 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -47,13 +47,10 @@ import org.elasticsearch.client.security.RefreshPolicy; import org.elasticsearch.client.tasks.TaskId; import org.elasticsearch.cluster.health.ClusterHealthStatus; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; -import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -63,6 +60,9 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.SuppressForbidden; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.rankeval.RankEvalRequest; @@ -422,6 +422,9 @@ static void addSearchRequestParams(Params params, SearchRequest searchRequest) { if (searchRequest.isCcsMinimizeRoundtrips() != SearchRequest.defaultCcsMinimizeRoundtrips(searchRequest)) { params.putParam("ccs_minimize_roundtrips", Boolean.toString(searchRequest.isCcsMinimizeRoundtrips())); } + if (searchRequest.isFieldsOptionEmulationEnabled() != SearchRequest.DEFAULT_FIELDS_EMULATION_ENABLED) { + params.putParam("enable_fields_emulation", Boolean.toString(searchRequest.isFieldsOptionEmulationEnabled())); + } if (searchRequest.getPreFilterShardSize() != null) { params.putParam("pre_filter_shard_size", Integer.toString(searchRequest.getPreFilterShardSize())); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index 00c4b1e2af7bf..88f47b1c412a6 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -52,17 +52,17 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.core.Tuple; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.MatchAllQueryBuilder; @@ -1353,6 +1353,8 @@ public void testMultiSearch() throws IOException { }); // scroll is not supported in the current msearch api, so unset it: searchRequest.scroll((Scroll) null); + // fields emulation is not supported in msearch api + searchRequest.setFieldsOptionEmulationEnabled(false); // only expand_wildcards, ignore_unavailable and allow_no_indices can be // specified from msearch api, so unset other options: IndicesOptions randomlyGenerated = searchRequest.indicesOptions(); @@ -2141,6 +2143,11 @@ private static void setRandomSearchParams(SearchRequest searchRequest, expectedParams.put("ccs_minimize_roundtrips", "false"); } } + if (randomBoolean()) { + boolean enableFieldsEmulation = randomBoolean(); + searchRequest.setFieldsOptionEmulationEnabled(enableFieldsEmulation); + expectedParams.put("enable_fields_emulation", Boolean.toString(enableFieldsEmulation)); + } if (randomBoolean()) { searchRequest.setMaxConcurrentShardRequests(randomIntBetween(1, Integer.MAX_VALUE)); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 5f833a85ebd91..42066a0e3dc13 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -91,7 +91,9 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla private String[] types = Strings.EMPTY_ARRAY; private boolean ccsMinimizeRoundtrips; - private boolean enableFieldsEmulation = false; + + public static final boolean DEFAULT_FIELDS_EMULATION_ENABLED = false; + private boolean enableFieldsEmulation = DEFAULT_FIELDS_EMULATION_ENABLED; @Nullable private final Version minCompatibleShardNode; @@ -444,7 +446,7 @@ public void setCcsMinimizeRoundtrips(boolean ccsMinimizeRoundtrips) { } /** - * Returns whether the "fields" option will be emulated via fetching from source on pre-7.10 nodes. + * Returns whether the "fields" option will be emulated via fetching from source on pre 7.10 nodes. * The default is false. */ public boolean isFieldsOptionEmulationEnabled() { @@ -452,7 +454,7 @@ public boolean isFieldsOptionEmulationEnabled() { } /** - * Sets whether the "fields" option will be emulated via fetching from source on pre-7.10 nodes. + * Sets whether the "fields" option will be emulated via fetching from source on pre 7.10 nodes. */ public void setFieldsOptionEmulationEnabled(boolean enableFieldsEmulation) { this.enableFieldsEmulation = enableFieldsEmulation; From 8f8f339e47434a58eb46652619241241c01e2c61 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 3 Aug 2021 16:41:41 +0200 Subject: [PATCH 12/31] fix tests --- .../org/elasticsearch/client/RequestConvertersTests.java | 4 +++- .../elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java | 6 ++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index 88f47b1c412a6..e89a6eccf0f9f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -2146,7 +2146,9 @@ private static void setRandomSearchParams(SearchRequest searchRequest, if (randomBoolean()) { boolean enableFieldsEmulation = randomBoolean(); searchRequest.setFieldsOptionEmulationEnabled(enableFieldsEmulation); - expectedParams.put("enable_fields_emulation", Boolean.toString(enableFieldsEmulation)); + if (enableFieldsEmulation == true) { + expectedParams.put("enable_fields_emulation", Boolean.toString(enableFieldsEmulation)); + } } if (randomBoolean()) { searchRequest.setMaxConcurrentShardRequests(randomIntBetween(1, Integer.MAX_VALUE)); diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index 3e3804689d34d..49cb792543408 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -235,10 +235,8 @@ public void testFieldsOptionEmulation() throws Exception { for (String minimizeRoundTrips : new String[] { "true", "false" }) { Request request = new Request("POST", "/_search"); request.addParameter("index", localIndex + "," + CLUSTER_ALIAS + ":" + remoteIndex); - if (UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_0_0)) { - request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); - request.addParameter("enable_fields_emulation", "true"); - } + request.addParameter("ccs_minimize_roundtrips", minimizeRoundTrips); + request.addParameter("enable_fields_emulation", "true"); request.setJsonEntity("{\"_source\": false, \"fields\": [\"*\"] , \"size\": " + expectedHitCount + "}"); Response response = lowLevelClient.performRequest(request); try ( From 2a4a82320ddef1aee4b3b5bc528ebbdf5eb3cecc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 3 Aug 2021 16:49:37 +0200 Subject: [PATCH 13/31] checkstyle --- .../java/org/elasticsearch/client/RequestConvertersTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index e89a6eccf0f9f..5405b4610b34b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -2146,7 +2146,7 @@ private static void setRandomSearchParams(SearchRequest searchRequest, if (randomBoolean()) { boolean enableFieldsEmulation = randomBoolean(); searchRequest.setFieldsOptionEmulationEnabled(enableFieldsEmulation); - if (enableFieldsEmulation == true) { + if (enableFieldsEmulation) { expectedParams.put("enable_fields_emulation", Boolean.toString(enableFieldsEmulation)); } } From 83811fecfb205761316926667bb38f0afcea0a3c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 4 Aug 2021 17:29:43 +0200 Subject: [PATCH 14/31] Add unit testing, pull out FieldsOptionSourceAdapter into its own class --- .../backwards/FieldsOptionEmulationIT.java | 12 +- .../resources/rest-api-spec/api/search.json | 5 + .../action/search/FetchSearchPhase.java | 3 +- .../search/FieldsOptionSourceAdapter.java | 132 ++++++++++++++++++ .../SearchQueryThenFetchAsyncAction.java | 7 +- .../action/search/TransportSearchAction.java | 123 ++-------------- .../search/WrappedQuerySearchResult.java | 1 - .../action/search/RestMultiSearchAction.java | 2 +- .../FieldsOptionSourceAdapterTests.java | 130 +++++++++++++++++ 9 files changed, 291 insertions(+), 124 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java create mode 100644 server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java index e7ce1f5f5bcf4..ab736f5b35725 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -75,11 +75,15 @@ private void createIndexOnNode(String indexName, String nodeName) throws IOExcep @SuppressWarnings("unchecked") public void testFieldOptionAdapterAllFields() throws Exception { - for (String includeSource : new String[] { "true", "false" }) { + for (String includeSource : new String[] { "true", "false", null }) { Request matchAllRequest = new Request("POST", "test_field_*/_search"); matchAllRequest.addParameter("enable_fields_emulation", "true"); - - matchAllRequest.setJsonEntity("{\"_source\":" + includeSource + " ,\"fields\":[\"*\"]}"); + if (includeSource == null) { + // omit _source parameter, this should behave the same as _source: true + matchAllRequest.setJsonEntity("{\"fields\":[\"*\"]}"); + } else { + matchAllRequest.setJsonEntity("{\"_source\":" + includeSource + " ,\"fields\":[\"*\"]}"); + } try ( RestClient client = buildClient( restClientSettings(), @@ -101,7 +105,7 @@ public void testFieldOptionAdapterAllFields() throws Exception { // if all nodes are > 7.10 we should get full "fields" output even for subfields assertTrue(((List) fieldsMap.get("test.keyword")).get(0).toString().startsWith("test_")); } - if (includeSource.equals("true")) { + if (includeSource == null || includeSource.equals("true")) { assertNotNull(hit.get("_source")); } else { assertNull(hit.get("_source")); diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index d127b188c7736..8f1155e34041b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -252,6 +252,11 @@ "min_compatible_shard_node":{ "type":"string", "description":"The minimum compatible version that all shards involved in search should have for this request to be successful" + }, + "enable_fields_emulation":{ + "type":"boolean", + "description":"Specify if fields option should be emulated on pre 7.10 nodes by fetching from source", + "default":"false" } }, "body":{ diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 3b423c12a7c75..ba3d7251865fb 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -13,7 +13,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.search.ScoreDoc; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.search.RescoreDocIds; @@ -168,7 +167,7 @@ private void executeFetch(final int shardIndex, final SearchShardTarget shardTar if (querySearchResult instanceof WrappedQuerySearchResult) { adapter = ((WrappedQuerySearchResult) querySearchResult).getAdapter(); } else { - adapter = TransportSearchAction.NOOP_FIELDSADAPTER; + adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; } context.getSearchTransport().sendExecuteFetch(connection, fetchSearchRequest, context.getTask(), new SearchActionListener(shardTarget, shardIndex) { diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java new file mode 100644 index 0000000000000..e784d69a8b343 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -0,0 +1,132 @@ +/* + * 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.automaton.CharacterRunAutomaton; +import org.elasticsearch.Version; +import org.elasticsearch.common.document.DocumentField; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.search.fetch.subphase.FieldAndFormat; +import org.elasticsearch.search.fetch.subphase.FieldFetcher; +import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.transport.Transport.Connection; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +interface FieldsOptionSourceAdapter { + default void adaptRequest(Consumer sourceConsumer) { + // noop + }; + default void adaptResponse(SearchHit[] searchHits) { + // noop + }; + + default boolean getRemoveSourceOnResponse() { + return false; + } + + String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + + "a search request that is targeting pre version 7.10 nodes."; + + static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuilder ccsSearchSource) { + Version version = connection.getVersion(); + if (version.before(Version.V_7_10_0)) { + List fetchFields = ccsSearchSource.fetchFields(); + if (fetchFields != null && fetchFields.isEmpty() == false) { + String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); + CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; + // We separate the "include_unmapped" field patters with wildcards from the rest in order to use less space in the + // lookup automaton + Map> partitions = fetchFields.stream() + .map(ff -> ff.field) + .collect(Collectors.partitioningBy((s -> Regex.isSimpleMatchPattern(s)))); + List unmappedWildcardPattern = partitions.get(true); + List unmappedConcreteFields = partitions.get(false); + if (unmappedWildcardPattern.isEmpty() == false) { + unmappedFieldsFetchAutomaton = new CharacterRunAutomaton( + Regex.simpleMatchToAutomaton(unmappedWildcardPattern.toArray(new String[unmappedWildcardPattern.size()])), + 100000 + ); + } + final FieldFetcher fieldFetcher = new FieldFetcher( + Collections.emptyMap(), + unmappedFieldsFetchAutomaton, + unmappedConcreteFields + ); + + FetchSourceContext fetchSource = ccsSearchSource.fetchSource(); + final boolean removeSourceOnResponse; + final SearchSourceBuilder adaptedSource; + + if (fetchSource == null || (fetchSource != null && fetchSource.fetchSource())) { + // case 1: original request has source: true, but no includes/exclude -> do nothing on request + if (fetchSource == null || fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { + // change nothing, we can get everything from source and can leave it when translating the response + removeSourceOnResponse = false; + adaptedSource = ccsSearchSource; + } else { + // original request has source includes/excludes set. In this case we don't want to silently + // overwrite the source parameter with something else, so we error instead + throw new IllegalArgumentException(FIELDS_EMULATION_ERROR_MSG); + } + } else { + // case 2: original request has source: false + adaptedSource = ccsSearchSource.shallowCopy(); + adaptedSource.fetchSource(new FetchSourceContext(true)); + adaptedSource.fetchSource(includes, null); + removeSourceOnResponse = true; + } + + return new FieldsOptionSourceAdapter() { + + @Override + public void adaptRequest(Consumer sourceConsumer) { + sourceConsumer.accept(adaptedSource); + } + + @Override + public void adaptResponse(SearchHit[] hits) { + for (SearchHit hit : hits) { + SourceLookup lookup = new SourceLookup(); + lookup.setSource(hit.getSourceAsMap()); + Map documentFields = Collections.emptyMap(); + try { + documentFields = fieldFetcher.fetch(lookup); + } catch (IOException e) { + // best effort fetching field, if this doesn't work continue + } + for (Map.Entry entry : documentFields.entrySet()) { + hit.setDocumentField(entry.getKey(), entry.getValue()); + } + if (removeSourceOnResponse) { + // original request didn't request source, so we remove it + hit.sourceRef(null); + } + } + } + + @Override + public boolean getRemoveSourceOnResponse() { + return removeSourceOnResponse; + } + }; + } + } + return FieldsOptionSourceAdapter.NOOP_ADAPTER; + } + + FieldsOptionSourceAdapter NOOP_ADAPTER = new FieldsOptionSourceAdapter() {}; +} \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index fecd5342e5b1b..e0f5419de8488 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.search.SearchPhaseResult; @@ -74,11 +73,11 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); final FieldsOptionSourceAdapter fieldsOptionAdapter; if (getRequest().isFieldsOptionEmulationEnabled()) { - fieldsOptionAdapter = TransportSearchAction.createFieldsOptionAdapter(connection, request.source()); + fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection, request.source()); } else { - fieldsOptionAdapter = TransportSearchAction.NOOP_FIELDSADAPTER; + fieldsOptionAdapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; } - fieldsOptionAdapter.adaptRequest(request.source(), request::source); + fieldsOptionAdapter.adaptRequest(request::source); getSearchTransport().sendExecuteQuery( connection, 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 2e073c5f8bd8b..11902581e13cc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -8,8 +8,6 @@ package org.elasticsearch.action.search; -import org.apache.lucene.util.automaton.CharacterRunAutomaton; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; @@ -32,11 +30,9 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.util.concurrent.AtomicArray; @@ -48,20 +44,15 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.ExecutorSelector; import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.fetch.subphase.FetchSourceContext; -import org.elasticsearch.search.fetch.subphase.FieldAndFormat; -import org.elasticsearch.search.fetch.subphase.FieldFetcher; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.lookup.SourceLookup; import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; import org.elasticsearch.tasks.Task; @@ -72,10 +63,8 @@ import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.Transport.Connection; import org.elasticsearch.transport.TransportService; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -91,7 +80,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.BiFunction; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongSupplier; import java.util.stream.Collectors; @@ -361,97 +349,6 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { source.collapse().getInnerHits().isEmpty(); } - public static FieldsOptionSourceAdapter createFieldsOptionAdapter(Connection connection, SearchSourceBuilder ccsSearchSource) { - Version version = connection.getVersion(); - if (version.before(Version.V_7_10_0)) { - List fetchFields = ccsSearchSource.fetchFields(); - if (fetchFields != null) { - if (fetchFields.isEmpty() == false) { - String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); - CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; - // We separate the "include_unmapped" field patters with wildcards from the rest in order to use less space in the - // lookup automaton - Map> partitions = fetchFields.stream() - .map(ff -> ff.field) - .collect(Collectors.partitioningBy((s -> Regex.isSimpleMatchPattern(s)))); - List unmappedWildcardPattern = partitions.get(true); - List unmappedConcreteFields = partitions.get(false); - if (unmappedWildcardPattern.isEmpty() == false) { - unmappedFieldsFetchAutomaton = new CharacterRunAutomaton( - Regex.simpleMatchToAutomaton(unmappedWildcardPattern.toArray(new String[unmappedWildcardPattern.size()])), - 100000 - ); - } - final FieldFetcher fieldFetcher = new FieldFetcher( - Collections.emptyMap(), - unmappedFieldsFetchAutomaton, - unmappedConcreteFields - ); - - return new FieldsOptionSourceAdapter() { - - private boolean removeSourceOnResponse = false; - - @Override - public void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { - FetchSourceContext fetchSource = source.fetchSource(); - // case 1: original request has source: true, but no includes/exclude -> do nothing on request - if (fetchSource != null && fetchSource.fetchSource()) { - if (fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { - // do nothing, we can get everything from source and can leave it when translating the response - } else { - // we probably need to error here - } - } - - // case 2: original request has source: false - if (fetchSource != null && fetchSource.fetchSource() == false) { - SearchSourceBuilder adaptedSource = source.shallowCopy(); - adaptedSource.fetchSource(new FetchSourceContext(true)); - adaptedSource.fetchSource(includes, null); - sourceConsumer.accept(adaptedSource); - removeSourceOnResponse = true; - } - } - - @Override - public void adaptResponse(SearchHit[] hits) { - for (SearchHit hit : hits) { - SourceLookup lookup = new SourceLookup(); - lookup.setSource(hit.getSourceAsMap()); - Map documentFields = Collections.emptyMap(); - try { - documentFields = fieldFetcher.fetch(lookup); - } catch (IOException e) { - // best effort fetching field, if this doesn't work continue - } - for (Map.Entry entry : documentFields.entrySet()) { - hit.setDocumentField(entry.getKey(), entry.getValue()); - } - if (removeSourceOnResponse) { - // original request didn't request source, so we remove it - hit.sourceRef(null); - } - } - } - }; - } - } - } - return NOOP_FIELDSADAPTER; - } - - interface FieldsOptionSourceAdapter { - default void adaptRequest(SearchSourceBuilder source, Consumer sourceConsumer) { - // noop - }; - default void adaptResponse(SearchHit[] searchHits) { - // noop - }; - } - - public static final FieldsOptionSourceAdapter NOOP_FIELDSADAPTER = new FieldsOptionSourceAdapter() {}; - static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, @@ -465,17 +362,17 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; if (searchRequest.isFieldsOptionEmulationEnabled()) { try { - adapter = createFieldsOptionAdapter(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); + adapter = FieldsOptionSourceAdapter.create(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); } catch (NoSuchRemoteClusterException ex) { // no connection version, adapter creation not possible if cluster not connected } } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + adapter.adaptRequest(ccsSearchRequest::source); final FieldsOptionSourceAdapter finalAdapter = adapter; Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); @@ -515,13 +412,15 @@ public void onFailure(Exception e) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = NOOP_FIELDSADAPTER; + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; if (searchRequest.isFieldsOptionEmulationEnabled()) { try { - - adapter = createFieldsOptionAdapter(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); + adapter = FieldsOptionSourceAdapter.create( + remoteClusterService.getConnection(clusterAlias), + searchRequest.source() + ); } catch (NoSuchRemoteClusterException ex) { - // don't create fields option converter in this case + // don't create fields option adapter in this case } } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( @@ -532,7 +431,7 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - adapter.adaptRequest(ccsSearchRequest.source(), ccsSearchRequest::source); + adapter.adaptRequest(ccsSearchRequest::source); ActionListener ccsListener = createCCSListener( clusterAlias, skipUnavailable, @@ -557,7 +456,7 @@ public void onFailure(Exception e) { searchResponseMerger, totalClusters, listener, - NOOP_FIELDSADAPTER + FieldsOptionSourceAdapter.NOOP_ADAPTER ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); diff --git a/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java b/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java index 32ef6e83c31cb..c711a6e4e8ab1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java @@ -9,7 +9,6 @@ package org.elasticsearch.action.search; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.action.search.TransportSearchAction.FieldsOptionSourceAdapter; import org.elasticsearch.common.io.stream.DelayableWriteable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 3946280d3cd35..3bdb28840c1db 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -17,13 +17,13 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.core.Tuple; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.Tuple; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestCancellableNodeClient; diff --git a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java new file mode 100644 index 0000000000000..2806239cbd73f --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java @@ -0,0 +1,130 @@ +/* + * 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.SetOnce; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.fetch.subphase.FetchSourceContext; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.transport.Transport; + +public class FieldsOptionSourceAdapterTests extends ESTestCase { + + /** + * test that on versions after 7.10 or when no fields are fetched, the returned adapter does nothing (is the noop adapter) + */ + public void testNoopFieldsAdapterCreation() { + SearchSourceBuilder source = new SearchSourceBuilder(); + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + assertSame(FieldsOptionSourceAdapter.NOOP_ADAPTER, adapter); + + adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_7_10_0, Version.CURRENT)), + source + ); + // no set fields should also return a noop adapter + assertSame(FieldsOptionSourceAdapter.NOOP_ADAPTER, adapter); + } + + /** + * test that with set fields pattern, the adapter does something. + * check correct request and response modifications + */ + public void testFieldsAdapterNoSource() { + // build adapter that gets all foo* fields + SearchSourceBuilder source = new SearchSourceBuilder().fetchField("foo*"); + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + + SetOnce rewrittenSource = new SetOnce<>(); + adapter.adaptRequest(rewrittenSource::set); + assertNull(rewrittenSource.get().fetchSource()); + assertFalse(adapter.getRemoveSourceOnResponse()); + + SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"foo\" : \"bar\", \"fuzz\":2, \"foo.bar\":[4,5]}")); + SearchHit[] searchHits = new SearchHit[] { hit }; + assertNull(hit.getFields().get("foo")); + assertNull(hit.getFields().get("foo.bar")); + assertNull(hit.getFields().get("fuzz")); + adapter.adaptResponse(searchHits); + assertEquals(1, hit.getFields().get("foo").getValues().size()); + assertEquals("bar", hit.getFields().get("foo").getValues().get(0)); + assertEquals(2, hit.getFields().get("foo.bar").getValues().size()); + assertEquals(4, hit.getFields().get("foo.bar").getValues().get(0)); + assertEquals(5, hit.getFields().get("foo.bar").getValues().get(1)); + assertNull(hit.getFields().get("fuzz")); + assertTrue(hit.hasSource()); + } + + /** + * Test that if request source is set to "false", we still set the source on the adapted request and remove it on return + */ + public void testFieldsAdapterSourceFalse() { + SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(false).fetchField("foo*"); + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + SetOnce rewrittenSource = new SetOnce<>(); + adapter.adaptRequest(rewrittenSource::set); + assertNotNull(rewrittenSource.get().fetchSource()); + assertTrue(rewrittenSource.get().fetchSource().fetchSource()); + assertEquals(1, rewrittenSource.get().fetchSource().includes().length); + assertEquals("foo*", rewrittenSource.get().fetchSource().includes()[0]); + assertTrue(adapter.getRemoveSourceOnResponse()); + + SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"foo\" : \"bar\", \"fuzz\":2, \"foo.bar\":[4,5]}")); + SearchHit[] searchHits = new SearchHit[] { hit }; + adapter.adaptResponse(searchHits); + assertFalse(hit.hasSource()); + } + + public void testFieldsAdapterSourceTrue() { + SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(true).fetchField("foo*"); + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + SetOnce rewrittenSource = new SetOnce<>(); + adapter.adaptRequest(rewrittenSource::set); + assertSame(source, rewrittenSource.get()); + assertFalse(adapter.getRemoveSourceOnResponse()); + + SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"foo\" : \"bar\", \"fuzz\":2, \"foo.bar\":[4,5]}")); + SearchHit[] searchHits = new SearchHit[] { hit }; + adapter.adaptResponse(searchHits); + assertTrue(hit.hasSource()); + } + + /** + * when _source is enabled with includes/excludes on the original request, we error + */ + public void testFieldsAdapterException() { + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + new SearchSourceBuilder().fetchField("foo*").fetchSource(new FetchSourceContext(true, new String[] { "bar" }, new String[0])) + )); + assertEquals(FieldsOptionSourceAdapter.FIELDS_EMULATION_ERROR_MSG, iae.getMessage()); + } + + private Transport.Connection mockConnection(Version version) { + DiscoveryNode node = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), version); + return new SearchAsyncActionTests.MockConnection(node); + } +} From 00c14c8f72a0760d5db1f6b7adf8079817b3029d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 5 Aug 2021 19:22:30 +0200 Subject: [PATCH 15/31] Adding testing around querying into ojects --- .../backwards/FieldsOptionEmulationIT.java | 21 +++++++++++++ .../FieldsOptionSourceAdapterTests.java | 30 +++++++++++++++++++ 2 files changed, 51 insertions(+) diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java index ab736f5b35725..a746d811f9a96 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/FieldsOptionEmulationIT.java @@ -179,4 +179,25 @@ public void testFieldOptionAdapterFilterFields() throws Exception { } } } + + @SuppressWarnings("unchecked") + public void testGettingObjects() throws Exception { + Request matchAllRequestFiltered = new Request("POST", + "test_field_*/_search"); + matchAllRequestFiltered.addParameter("enable_fields_emulation", "true"); + matchAllRequestFiltered.setJsonEntity("{\"_source\":false,\"fields\":[\"obj\"]}"); + try ( + RestClient client = buildClient(restClientSettings(), newNodes.stream().map(Node::getPublishAddress).toArray(HttpHost[]::new)) + ) { + Response response = client.performRequest(matchAllRequestFiltered); + ObjectPath responseObject = ObjectPath.createFromResponse(response); + List> hits = responseObject.evaluate("hits.hits"); + assertEquals(10, hits.size()); + for (Map hit : hits) { + // getting a root object shouldn't return anything vie the 'fields' api + Map fieldsMap = (Map) hit.get("fields"); + assertNull(fieldsMap); + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java index 2806239cbd73f..dcb091cb44d43 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java @@ -112,6 +112,36 @@ public void testFieldsAdapterSourceTrue() { assertTrue(hit.hasSource()); } + /** + * while querying the root or object fields works in "_source" filtering, we should not + * return this via the fields API, since our regular lookup there only returns flattened leaf values + */ + public void testFieldsAdapterObjects() { + SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(false).fetchField("obj"); + FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + + SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"obj\": { \"foo\" : \"value\"}}")); + SearchHit[] searchHits = new SearchHit[] { hit }; + adapter.adaptResponse(searchHits); + assertNull(hit.getFields().get("obj")); + + source = new SearchSourceBuilder().fetchSource(false).fetchField("obj.*"); + adapter = FieldsOptionSourceAdapter.create( + mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), + source + ); + + hit = new SearchHit(1).sourceRef(new BytesArray("{ \"obj\": { \"foo\" : \"value\"}}")); + searchHits = new SearchHit[] { hit }; + adapter.adaptResponse(searchHits); + assertNull(hit.getFields().get("obj")); + assertEquals(1, hit.getFields().get("obj.foo").getValues().size()); + assertEquals("value", hit.getFields().get("obj.foo").getValues().get(0)); + } + /** * when _source is enabled with includes/excludes on the original request, we error */ From c7d90cf61f708a8f2f8a88da2fdfc6d1e25cdcfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 30 Aug 2021 16:15:43 +0200 Subject: [PATCH 16/31] Update request serialization logic --- .../java/org/elasticsearch/action/search/SearchRequest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 42066a0e3dc13..ad65a929549f6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -250,7 +250,7 @@ public SearchRequest(StreamInput in) throws IOException { } else { minCompatibleShardNode = null; } - if (in.getVersion().onOrAfter(Version.V_7_15_0)) { + if (in.getVersion().onOrAfter(Version.V_7_16_0)) { this.enableFieldsEmulation = in.readBoolean(); } else { this.enableFieldsEmulation = false; @@ -295,7 +295,7 @@ public void writeTo(StreamOutput out) throws IOException { Version.writeVersion(minCompatibleShardNode, out); } } - if (out.getVersion().onOrAfter(Version.V_7_15_0)) { + if (out.getVersion().onOrAfter(Version.V_7_16_0)) { out.writeBoolean(enableFieldsEmulation); } } From 3e5ff121427a878d70e7e7071ac3495af29c4240 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 3 Sep 2021 16:26:21 +0200 Subject: [PATCH 17/31] WIP refactoring test infra --- .../upgrades/CCSFieldsOptionEmulationIT.java | 151 ++---------------- .../upgrades/SearchStatesIT.java | 140 ++-------------- .../test/rest/AbstractCCSRestTestCase.java | 150 +++++++++++++++++ 3 files changed, 181 insertions(+), 260 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/test/rest/AbstractCCSRestTestCase.java diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index 49cb792543408..3f3ca8cf9bb80 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -31,7 +31,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.search.SearchResponse; @@ -52,153 +51,24 @@ import org.elasticsearch.rest.action.document.RestIndexAction; import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; -import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.test.rest.yaml.ObjectPath; +import org.elasticsearch.test.rest.AbstractCCSRestTestCase; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.not; /** * This test ensure that we emulate the "fields" option when the local cluster supports it but the remote * cluster is running an older compatible version. */ -public class CCSFieldsOptionEmulationIT extends ESRestTestCase { +public class CCSFieldsOptionEmulationIT extends AbstractCCSRestTestCase { private static final Logger LOGGER = LogManager.getLogger(CCSFieldsOptionEmulationIT.class); private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); private static final String CLUSTER_ALIAS = "remote_cluster"; - // TODO refactor this and SearchStateIT so that common code moves somewhere else - static class Node { - final String id; - final String name; - final Version version; - final String transportAddress; - final String httpAddress; - final Map attributes; - - Node(String id, String name, Version version, String transportAddress, String httpAddress, Map attributes) { - this.id = id; - this.name = name; - this.version = version; - this.transportAddress = transportAddress; - this.httpAddress = httpAddress; - this.attributes = attributes; - } - - @Override - public String toString() { - return "Node{" + - "id='" + id + '\'' + - ", name='" + name + '\'' + - ", version=" + version + - ", transportAddress='" + transportAddress + '\'' + - ", httpAddress='" + httpAddress + '\'' + - ", attributes=" + attributes + - '}'; - } - } - - static List getNodes(RestClient restClient) throws IOException { - Response response = restClient.performRequest(new Request("GET", "_nodes")); - ObjectPath objectPath = ObjectPath.createFromResponse(response); - final Map nodeMap = objectPath.evaluate("nodes"); - final List nodes = new ArrayList<>(); - for (String id : nodeMap.keySet()) { - final String name = objectPath.evaluate("nodes." + id + ".name"); - final Version version = Version.fromString(objectPath.evaluate("nodes." + id + ".version")); - final String transportAddress = objectPath.evaluate("nodes." + id + ".transport.publish_address"); - final String httpAddress = objectPath.evaluate("nodes." + id + ".http.publish_address"); - final Map attributes = objectPath.evaluate("nodes." + id + ".attributes"); - nodes.add(new Node(id, name, version, transportAddress, httpAddress, attributes)); - } - return nodes; - } - - static List parseHosts(String props) { - final String address = System.getProperty(props); - assertNotNull("[" + props + "] is not configured", address); - String[] stringUrls = address.split(","); - List hosts = new ArrayList<>(stringUrls.length); - for (String stringUrl : stringUrls) { - int portSeparator = stringUrl.lastIndexOf(':'); - if (portSeparator < 0) { - throw new IllegalArgumentException("Illegal cluster url [" + stringUrl + "]"); - } - String host = stringUrl.substring(0, portSeparator); - int port = Integer.parseInt(stringUrl.substring(portSeparator + 1)); - hosts.add(new HttpHost(host, port, "http")); - } - assertThat("[" + props + "] is empty", hosts, not(empty())); - return hosts; - } - - public static void configureRemoteClusters(List remoteNodes) throws Exception { - assertThat(remoteNodes, hasSize(2)); - final String remoteClusterSettingPrefix = "cluster.remote." + CLUSTER_ALIAS + "."; - try (RestHighLevelClient localClient = newLocalClient()) { - final Settings remoteConnectionSettings; - if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0) || randomBoolean()) { - final List seeds = remoteNodes.stream() - .filter(n -> n.attributes.containsKey("gateway")) - .map(n -> n.transportAddress) - .collect(Collectors.toList()); - assertThat(seeds, hasSize(2)); - LOGGER.info("--> use sniff mode with seed [{}], remote nodes [{}]", seeds, remoteNodes); - if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0)) { - remoteConnectionSettings = Settings.builder() - .putList(remoteClusterSettingPrefix + "seeds", seeds) - .build(); - } else { - remoteConnectionSettings = Settings.builder() - .putNull(remoteClusterSettingPrefix + "proxy_address") - .put(remoteClusterSettingPrefix + "mode", "sniff") - .putList(remoteClusterSettingPrefix + "seeds", seeds) - .build(); - } - } else { - final Node proxyNode = randomFrom(remoteNodes); - LOGGER.info("--> use proxy node [{}], remote nodes [{}]", proxyNode, remoteNodes); - remoteConnectionSettings = Settings.builder() - .putNull(remoteClusterSettingPrefix + "seeds") - .put(remoteClusterSettingPrefix + "mode", "proxy") - .put(remoteClusterSettingPrefix + "proxy_address", proxyNode.transportAddress) - .build(); - } - assertTrue( - localClient.cluster() - .putSettings(new ClusterUpdateSettingsRequest().persistentSettings(remoteConnectionSettings), RequestOptions.DEFAULT) - .isAcknowledged() - ); - assertBusy(() -> { - final Response resp = localClient.getLowLevelClient().performRequest(new Request("GET", "/_remote/info")); - assertOK(resp); - final ObjectPath objectPath = ObjectPath.createFromResponse(resp); - assertNotNull(objectPath.evaluate(CLUSTER_ALIAS)); - assertTrue(objectPath.evaluate(CLUSTER_ALIAS + ".connected")); - }, 60, TimeUnit.SECONDS); - } - } - - static RestHighLevelClient newLocalClient() { - final List hosts = parseHosts("tests.rest.cluster"); - final int index = random().nextInt(hosts.size()); - LOGGER.info("Using client node {}", index); - return new RestHighLevelClient(RestClient.builder(hosts.get(index))); - } - - static RestHighLevelClient newRemoteClient() { - return new RestHighLevelClient(RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster")))); - } - static int indexDocs(RestHighLevelClient client, String index, int numDocs, boolean expectWarnings) throws IOException { for (int i = 0; i < numDocs; i++) { Request indexDoc = new Request("PUT", index + "/type/" + i); @@ -212,10 +82,21 @@ static int indexDocs(RestHighLevelClient client, String index, int numDocs, bool return numDocs; } + static RestHighLevelClient newLocalClient(Logger logger) { + final List hosts = parseHosts("tests.rest.cluster"); + final int index = random().nextInt(hosts.size()); + logger.info("Using client node {}", index); + return new RestHighLevelClient(RestClient.builder(hosts.get(index))); + } + + static RestHighLevelClient newRemoteClient() { + return new RestHighLevelClient(RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster")))); + } + public void testFieldsOptionEmulation() throws Exception { String localIndex = "test_bwc_fields_index"; String remoteIndex = "test_bwc_fields_remote_index"; - try (RestHighLevelClient localClient = newLocalClient(); + try (RestHighLevelClient localClient = newLocalClient(LOGGER); RestHighLevelClient remoteClient = newRemoteClient()) { localClient.indices().create(new CreateIndexRequest(localIndex) .settings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))), @@ -230,7 +111,9 @@ public void testFieldsOptionEmulation() throws Exception { int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 20), expectRemoteIndexWarnings); int expectedHitCount = localNumDocs + remoteNumDocs; - configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); + List remoteNodes = getNodes(remoteClient.getLowLevelClient()); + assertThat(remoteNodes, hasSize(2)); + configureRemoteClusters(getNodes(remoteClient.getLowLevelClient()), CLUSTER_ALIAS, UPGRADE_FROM_VERSION, LOGGER); RestClient lowLevelClient = localClient.getLowLevelClient(); for (String minimizeRoundTrips : new String[] { "true", "false" }) { Request request = new Request("POST", "/_search"); 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 8e7f3a399e3df..90a943a31c342 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 @@ -31,7 +31,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.search.SearchResponse; @@ -49,150 +48,33 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.action.document.RestIndexAction; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; -import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.test.rest.yaml.ObjectPath; +import org.elasticsearch.test.rest.AbstractCCSRestTestCase; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.ArrayList; import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.not; /** * This test ensure that we keep the search states of a CCS request correctly when the local and remote clusters * have different but compatible versions. See SearchService#createAndPutReaderContext */ -public class SearchStatesIT extends ESRestTestCase { +public class SearchStatesIT extends AbstractCCSRestTestCase { private static final Logger LOGGER = LogManager.getLogger(SearchStatesIT.class); private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); private static final String CLUSTER_ALIAS = "remote_cluster"; - static class Node { - final String id; - final String name; - final Version version; - final String transportAddress; - final String httpAddress; - final Map attributes; - - Node(String id, String name, Version version, String transportAddress, String httpAddress, Map attributes) { - this.id = id; - this.name = name; - this.version = version; - this.transportAddress = transportAddress; - this.httpAddress = httpAddress; - this.attributes = attributes; - } - - @Override - public String toString() { - return "Node{" + - "id='" + id + '\'' + - ", name='" + name + '\'' + - ", version=" + version + - ", transportAddress='" + transportAddress + '\'' + - ", httpAddress='" + httpAddress + '\'' + - ", attributes=" + attributes + - '}'; - } - } - - static List getNodes(RestClient restClient) throws IOException { - Response response = restClient.performRequest(new Request("GET", "_nodes")); - ObjectPath objectPath = ObjectPath.createFromResponse(response); - final Map nodeMap = objectPath.evaluate("nodes"); - final List nodes = new ArrayList<>(); - for (String id : nodeMap.keySet()) { - final String name = objectPath.evaluate("nodes." + id + ".name"); - final Version version = Version.fromString(objectPath.evaluate("nodes." + id + ".version")); - final String transportAddress = objectPath.evaluate("nodes." + id + ".transport.publish_address"); - final String httpAddress = objectPath.evaluate("nodes." + id + ".http.publish_address"); - final Map attributes = objectPath.evaluate("nodes." + id + ".attributes"); - nodes.add(new Node(id, name, version, transportAddress, httpAddress, attributes)); - } - return nodes; - } - - static List parseHosts(String props) { - final String address = System.getProperty(props); - assertNotNull("[" + props + "] is not configured", address); - String[] stringUrls = address.split(","); - List hosts = new ArrayList<>(stringUrls.length); - for (String stringUrl : stringUrls) { - int portSeparator = stringUrl.lastIndexOf(':'); - if (portSeparator < 0) { - throw new IllegalArgumentException("Illegal cluster url [" + stringUrl + "]"); - } - String host = stringUrl.substring(0, portSeparator); - int port = Integer.parseInt(stringUrl.substring(portSeparator + 1)); - hosts.add(new HttpHost(host, port, "http")); - } - assertThat("[" + props + "] is empty", hosts, not(empty())); - return hosts; - } - - public static void configureRemoteClusters(List remoteNodes) throws Exception { - assertThat(remoteNodes, hasSize(3)); - final String remoteClusterSettingPrefix = "cluster.remote." + CLUSTER_ALIAS + "."; - try (RestHighLevelClient localClient = newLocalClient()) { - final Settings remoteConnectionSettings; - if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0) || randomBoolean()) { - final List seeds = remoteNodes.stream() - .filter(n -> n.attributes.containsKey("gateway")) - .map(n -> n.transportAddress) - .collect(Collectors.toList()); - assertThat(seeds, hasSize(2)); - LOGGER.info("--> use sniff mode with seed [{}], remote nodes [{}]", seeds, remoteNodes); - if (UPGRADE_FROM_VERSION.before(Version.V_7_6_0)) { - remoteConnectionSettings = Settings.builder() - .putList(remoteClusterSettingPrefix + "seeds", seeds) - .build(); - } else { - remoteConnectionSettings = Settings.builder() - .putNull(remoteClusterSettingPrefix + "proxy_address") - .put(remoteClusterSettingPrefix + "mode", "sniff") - .putList(remoteClusterSettingPrefix + "seeds", seeds) - .build(); - } - } else { - final Node proxyNode = randomFrom(remoteNodes); - LOGGER.info("--> use proxy node [{}], remote nodes [{}]", proxyNode, remoteNodes); - remoteConnectionSettings = Settings.builder() - .putNull(remoteClusterSettingPrefix + "seeds") - .put(remoteClusterSettingPrefix + "mode", "proxy") - .put(remoteClusterSettingPrefix + "proxy_address", proxyNode.transportAddress) - .build(); - } - assertTrue( - localClient.cluster() - .putSettings(new ClusterUpdateSettingsRequest().persistentSettings(remoteConnectionSettings), RequestOptions.DEFAULT) - .isAcknowledged() - ); - assertBusy(() -> { - final Response resp = localClient.getLowLevelClient().performRequest(new Request("GET", "/_remote/info")); - assertOK(resp); - final ObjectPath objectPath = ObjectPath.createFromResponse(resp); - assertNotNull(objectPath.evaluate(CLUSTER_ALIAS)); - assertTrue(objectPath.evaluate(CLUSTER_ALIAS + ".connected")); - }, 60, TimeUnit.SECONDS); - } - } - - static RestHighLevelClient newLocalClient() { + protected static RestHighLevelClient newLocalClient(Logger logger) { final List hosts = parseHosts("tests.rest.cluster"); final int index = random().nextInt(hosts.size()); - LOGGER.info("Using client node {}", index); + logger.info("Using client node {}", index); return new RestHighLevelClient(RestClient.builder(hosts.get(index))); } - static RestHighLevelClient newRemoteClient() { + protected static RestHighLevelClient newRemoteClient() { return new RestHighLevelClient(RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster")))); } @@ -208,7 +90,7 @@ static int indexDocs(RestHighLevelClient client, String index, int numDocs) thro } void verifySearch(String localIndex, int localNumDocs, String remoteIndex, int remoteNumDocs) { - try (RestHighLevelClient localClient = newLocalClient()) { + try (RestHighLevelClient localClient = newLocalClient(LOGGER)) { Request request = new Request("POST", "/_search"); final int expectedDocs; if (randomBoolean()) { @@ -239,7 +121,7 @@ void verifySearch(String localIndex, int localNumDocs, String remoteIndex, int r public void testBWCSearchStates() throws Exception { String localIndex = "test_bwc_search_states_index"; String remoteIndex = "test_bwc_search_states_remote_index"; - try (RestHighLevelClient localClient = newLocalClient(); + try (RestHighLevelClient localClient = newLocalClient(LOGGER); RestHighLevelClient remoteClient = newRemoteClient()) { localClient.indices().create(new CreateIndexRequest(localIndex) .settings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))), @@ -251,7 +133,13 @@ public void testBWCSearchStates() throws Exception { RequestOptions.DEFAULT); int remoteNumDocs = indexDocs(remoteClient, remoteIndex, between(10, 100)); - configureRemoteClusters(getNodes(remoteClient.getLowLevelClient())); + List remoteNodes = getNodes(remoteClient.getLowLevelClient()); + assertThat(remoteNodes, hasSize(3)); + List seeds = remoteNodes.stream().filter(n -> n.attributes.containsKey("gateway")) + .map(n -> n.transportAddress) + .collect(Collectors.toList()); + assertThat(seeds, hasSize(2)); + configureRemoteClusters(remoteNodes, CLUSTER_ALIAS, UPGRADE_FROM_VERSION, LOGGER); int iterations = between(1, 20); for (int i = 0; i < iterations; i++) { verifySearch(localIndex, localNumDocs, CLUSTER_ALIAS + ":" + remoteIndex, remoteNumDocs); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/AbstractCCSRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/AbstractCCSRestTestCase.java new file mode 100644 index 0000000000000..3b93569ad8b1e --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/AbstractCCSRestTestCase.java @@ -0,0 +1,150 @@ +/* + * 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.test.rest; + +import org.apache.http.HttpHost; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.rest.yaml.ObjectPath; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.not; + +public abstract class AbstractCCSRestTestCase extends ESRestTestCase { + + protected static class Node { + public final String id; + public final String name; + public final Version version; + public final String transportAddress; + public final String httpAddress; + public final Map attributes; + + Node(String id, String name, Version version, String transportAddress, String httpAddress, Map attributes) { + this.id = id; + this.name = name; + this.version = version; + this.transportAddress = transportAddress; + this.httpAddress = httpAddress; + this.attributes = attributes; + } + + @Override + public String toString() { + return "Node{" + + "id='" + id + '\'' + + ", name='" + name + '\'' + + ", version=" + version + + ", transportAddress='" + transportAddress + '\'' + + ", httpAddress='" + httpAddress + '\'' + + ", attributes=" + attributes + + '}'; + } + } + + protected static List getNodes(RestClient restClient) throws IOException { + Response response = restClient.performRequest(new Request("GET", "_nodes")); + ObjectPath objectPath = ObjectPath.createFromResponse(response); + final Map nodeMap = objectPath.evaluate("nodes"); + final List nodes = new ArrayList<>(); + for (String id : nodeMap.keySet()) { + final String name = objectPath.evaluate("nodes." + id + ".name"); + final Version version = Version.fromString(objectPath.evaluate("nodes." + id + ".version")); + final String transportAddress = objectPath.evaluate("nodes." + id + ".transport.publish_address"); + final String httpAddress = objectPath.evaluate("nodes." + id + ".http.publish_address"); + final Map attributes = objectPath.evaluate("nodes." + id + ".attributes"); + nodes.add(new Node(id, name, version, transportAddress, httpAddress, attributes)); + } + return nodes; + } + + protected static List parseHosts(String props) { + final String address = System.getProperty(props); + assertNotNull("[" + props + "] is not configured", address); + String[] stringUrls = address.split(","); + List hosts = new ArrayList<>(stringUrls.length); + for (String stringUrl : stringUrls) { + int portSeparator = stringUrl.lastIndexOf(':'); + if (portSeparator < 0) { + throw new IllegalArgumentException("Illegal cluster url [" + stringUrl + "]"); + } + String host = stringUrl.substring(0, portSeparator); + int port = Integer.parseInt(stringUrl.substring(portSeparator + 1)); + hosts.add(new HttpHost(host, port, "http")); + } + assertThat("[" + props + "] is empty", hosts, not(empty())); + return hosts; + } + + private static RestClient createLocalClient(Logger logger) { + final List hosts = parseHosts("tests.rest.cluster"); + final int index = random().nextInt(hosts.size()); + logger.info("Using client node {}", index); + return RestClient.builder(hosts.get(index)).build(); + } + + public static void configureRemoteClusters(List remoteNodes, String remoteClusterAlias, Version upgradeFromVersion, Logger logger) + throws Exception { + final String remoteClusterSettingPrefix = "cluster.remote." + remoteClusterAlias + "."; + try (RestClient localClient = createLocalClient(logger)) { + final Settings remoteConnectionSettings; + if (upgradeFromVersion.before(Version.V_7_6_0) || randomBoolean()) { + final List seeds = remoteNodes.stream() + .filter(n -> n.attributes.containsKey("gateway")) + .map(n -> n.transportAddress) + .collect(Collectors.toList()); + logger.info("--> use sniff mode with seed [{}], remote nodes [{}]", seeds, remoteNodes); + if (upgradeFromVersion.before(Version.V_7_6_0)) { + remoteConnectionSettings = Settings.builder().putList(remoteClusterSettingPrefix + "seeds", seeds).build(); + } else { + remoteConnectionSettings = Settings.builder() + .putNull(remoteClusterSettingPrefix + "proxy_address") + .put(remoteClusterSettingPrefix + "mode", "sniff") + .putList(remoteClusterSettingPrefix + "seeds", seeds) + .build(); + } + } else { + final Node proxyNode = randomFrom(remoteNodes); + logger.info("--> use proxy node [{}], remote nodes [{}]", proxyNode, remoteNodes); + remoteConnectionSettings = Settings.builder() + .putNull(remoteClusterSettingPrefix + "seeds") + .put(remoteClusterSettingPrefix + "mode", "proxy") + .put(remoteClusterSettingPrefix + "proxy_address", proxyNode.transportAddress) + .build(); + } + ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest().persistentSettings(remoteConnectionSettings); + assertBusy(() -> { + Request request = new Request("PUT", "/_cluster/settings"); + request.setJsonEntity(Strings.toString(settingsRequest)); + final Response resp = localClient.performRequest(request); + assertOK(resp); + }, 60, TimeUnit.SECONDS); + assertBusy(() -> { + final Response resp = localClient.performRequest(new Request("GET", "/_remote/info")); + assertOK(resp); + final ObjectPath objectPath = ObjectPath.createFromResponse(resp); + assertNotNull(objectPath.evaluate(remoteClusterAlias)); + assertTrue(objectPath.evaluate(remoteClusterAlias + ".connected")); + }, 60, TimeUnit.SECONDS); + } + } +} From 55343f0910fbe7d632d347b9f458a2c5e139dd11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 6 Sep 2021 17:52:52 +0200 Subject: [PATCH 18/31] Add test for multivalued and object fields --- .../upgrades/CCSFieldsOptionEmulationIT.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index 3f3ca8cf9bb80..611e6744ce184 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -72,7 +72,7 @@ public class CCSFieldsOptionEmulationIT extends AbstractCCSRestTestCase { static int indexDocs(RestHighLevelClient client, String index, int numDocs, boolean expectWarnings) throws IOException { for (int i = 0; i < numDocs; i++) { Request indexDoc = new Request("PUT", index + "/type/" + i); - indexDoc.setJsonEntity("{\"f\":" + i + "}"); + indexDoc.setJsonEntity("{\"field\":" + i + ", \"array\": [1, 2, 3] , \"obj\": { \"innerObj\" : \"foo\" } }"); if (expectWarnings) { indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); } @@ -137,7 +137,13 @@ public void testFieldsOptionEmulation() throws Exception { assertFalse("No source in hit expected but was: " + hit.toString(), hit.hasSource()); Map fields = hit.getFields(); assertNotNull(fields); - assertNotNull("Field `f` not found, hit was: " + hit.toString(), fields.get("f")); + assertNotNull("Field `field` not found, hit was: " + hit.toString(), fields.get("field")); + DocumentField arrayField = fields.get("array"); + assertNotNull("Field `array` not found, hit was: " + hit.toString(), arrayField); + assertEquals(3, ((List) arrayField.getValues()).size()); + assertNull("Object fields should be flattened by the fields API", fields.get("obj")); + assertEquals(1, fields.get("obj.innerObj").getValues().size()); + assertEquals("foo", fields.get("obj.innerObj").getValue()); } } } From 7faf8fc2f2c251f0babc39ebbe8bfda342743c77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 8 Sep 2021 12:42:39 +0200 Subject: [PATCH 19/31] rename method arg and remove 'enable' option from rest spec --- .../src/main/resources/rest-api-spec/api/search.json | 7 +------ .../action/search/FieldsOptionSourceAdapter.java | 10 +++++----- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 8f1155e34041b..3d844220ba7dd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -252,12 +252,7 @@ "min_compatible_shard_node":{ "type":"string", "description":"The minimum compatible version that all shards involved in search should have for this request to be successful" - }, - "enable_fields_emulation":{ - "type":"boolean", - "description":"Specify if fields option should be emulated on pre 7.10 nodes by fetching from source", - "default":"false" - } + } }, "body":{ "description":"The search definition using the Query DSL" diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java index e784d69a8b343..ea146325c611b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -41,10 +41,10 @@ default boolean getRemoveSourceOnResponse() { String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + "a search request that is targeting pre version 7.10 nodes."; - static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuilder ccsSearchSource) { + static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuilder searchSource) { Version version = connection.getVersion(); if (version.before(Version.V_7_10_0)) { - List fetchFields = ccsSearchSource.fetchFields(); + List fetchFields = searchSource.fetchFields(); if (fetchFields != null && fetchFields.isEmpty() == false) { String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; @@ -67,7 +67,7 @@ static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuild unmappedConcreteFields ); - FetchSourceContext fetchSource = ccsSearchSource.fetchSource(); + FetchSourceContext fetchSource = searchSource.fetchSource(); final boolean removeSourceOnResponse; final SearchSourceBuilder adaptedSource; @@ -76,7 +76,7 @@ static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuild if (fetchSource == null || fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { // change nothing, we can get everything from source and can leave it when translating the response removeSourceOnResponse = false; - adaptedSource = ccsSearchSource; + adaptedSource = searchSource; } else { // original request has source includes/excludes set. In this case we don't want to silently // overwrite the source parameter with something else, so we error instead @@ -84,7 +84,7 @@ static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuild } } else { // case 2: original request has source: false - adaptedSource = ccsSearchSource.shallowCopy(); + adaptedSource = searchSource.shallowCopy(); adaptedSource.fetchSource(new FetchSourceContext(true)); adaptedSource.fetchSource(includes, null); removeSourceOnResponse = true; From c65b4e5640d2637dc2f55d16dde25401d13b234d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 8 Sep 2021 17:27:50 +0200 Subject: [PATCH 20/31] Add test for multi-field behaviour --- .../upgrades/CCSFieldsOptionEmulationIT.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java index 611e6744ce184..53293febca983 100644 --- a/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java +++ b/qa/ccs-old-version-remote-cluster/src/test/java/org/elasticsearch/upgrades/CCSFieldsOptionEmulationIT.java @@ -72,7 +72,7 @@ public class CCSFieldsOptionEmulationIT extends AbstractCCSRestTestCase { static int indexDocs(RestHighLevelClient client, String index, int numDocs, boolean expectWarnings) throws IOException { for (int i = 0; i < numDocs; i++) { Request indexDoc = new Request("PUT", index + "/type/" + i); - indexDoc.setJsonEntity("{\"field\":" + i + ", \"array\": [1, 2, 3] , \"obj\": { \"innerObj\" : \"foo\" } }"); + indexDoc.setJsonEntity("{\"field\": \"f" + i + "\", \"array\": [1, 2, 3] , \"obj\": { \"innerObj\" : \"foo\" } }"); if (expectWarnings) { indexDoc.setOptions(expectWarnings(RestIndexAction.TYPES_DEPRECATION_MESSAGE)); } @@ -138,6 +138,15 @@ public void testFieldsOptionEmulation() throws Exception { Map fields = hit.getFields(); assertNotNull(fields); assertNotNull("Field `field` not found, hit was: " + hit.toString(), fields.get("field")); + if (hit.getIndex().equals(localIndex) || UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_10_0)) { + assertNotNull("Field `field.keyword` not found, hit was: " + hit.toString(), fields.get("field.keyword")); + } else { + // we won't be able to get multi-field for remote indices below V7.10 + assertNull( + "Field `field.keyword` should not be returned, hit was: " + hit.toString(), + fields.get("field.keyword") + ); + } DocumentField arrayField = fields.get("array"); assertNotNull("Field `array` not found, hit was: " + hit.toString(), arrayField); assertEquals(3, ((List) arrayField.getValues()).size()); From 8915654af439a7eea7ba26f328e5a71c365784e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 9 Sep 2021 12:45:41 +0200 Subject: [PATCH 21/31] Remove need for wrapping query result --- .../action/search/FetchSearchPhase.java | 5 +- .../SearchQueryThenFetchAsyncAction.java | 31 +- .../search/WrappedQuerySearchResult.java | 326 ------------------ 3 files changed, 7 insertions(+), 355 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index ba3d7251865fb..ae5726773d3d2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -164,11 +164,12 @@ private void executeFetch(final int shardIndex, final SearchShardTarget shardTar final ShardFetchSearchRequest fetchSearchRequest, final QuerySearchResult querySearchResult, final Transport.Connection connection) { final FieldsOptionSourceAdapter adapter; - if (querySearchResult instanceof WrappedQuerySearchResult) { - adapter = ((WrappedQuerySearchResult) querySearchResult).getAdapter(); + if (context.getRequest().isFieldsOptionEmulationEnabled()) { + adapter = FieldsOptionSourceAdapter.create(connection, context.getRequest().source()); } else { adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; } + context.getSearchTransport().sendExecuteFetch(connection, fetchSearchRequest, context.getTask(), new SearchActionListener(shardTarget, shardIndex) { @Override diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index e0f5419de8488..f74785293e4ef 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -71,34 +71,11 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final SearchActionListener listener) { ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); - final FieldsOptionSourceAdapter fieldsOptionAdapter; if (getRequest().isFieldsOptionEmulationEnabled()) { - fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection, request.source()); - } else { - fieldsOptionAdapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; - } - fieldsOptionAdapter.adaptRequest(request::source); - - getSearchTransport().sendExecuteQuery( - connection, - request, - getTask(), - new SearchActionListener(shard, listener.requestIndex) { - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - - @Override - protected void innerOnResponse(SearchPhaseResult response) { - if (response instanceof QuerySearchResult) { - response = new WrappedQuerySearchResult((QuerySearchResult) response, fieldsOptionAdapter); - } - listener.onResponse(response); - } - } - ); + FieldsOptionSourceAdapter fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection, request.source()); + fieldsOptionAdapter.adaptRequest(request::source); + } + getSearchTransport().sendExecuteQuery(connection, request, getTask(), listener); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java b/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java deleted file mode 100644 index c711a6e4e8ab1..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/WrappedQuerySearchResult.java +++ /dev/null @@ -1,326 +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.search.TotalHits; -import org.elasticsearch.common.io.stream.DelayableWriteable; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.RescoreDocIds; -import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.internal.ShardSearchContextId; -import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.search.profile.ProfileShardResult; -import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.suggest.Suggest; - -import java.io.IOException; - -/** - * Wraps a {@link QuerySearchResult} together with an adapter that can be used to - * translate calls to pre 7.10 nodes when using the "fields" option and to translate - * the resulting response back - */ -final class WrappedQuerySearchResult extends QuerySearchResult { - - private final FieldsOptionSourceAdapter adapter; - private final QuerySearchResult original; - - WrappedQuerySearchResult(QuerySearchResult original, FieldsOptionSourceAdapter adapter) { - this.original = original; - this.adapter = adapter; - } - - /** - * @return the adapter to use or null if no adaption is necessary - */ - FieldsOptionSourceAdapter getAdapter() { - return this.adapter; - } - - @Override - public void remoteAddress(TransportAddress remoteAddress) { - original.remoteAddress(remoteAddress); - } - - @Override - public TransportAddress remoteAddress() { - return original.remoteAddress(); - } - - @Override - public void incRef() { - original.incRef(); - } - - @Override - public boolean tryIncRef() { - return original.tryIncRef(); - } - - @Override - public boolean decRef() { - return original.decRef(); - } - - @Override - public int hashCode() { - return original.hashCode(); - } - - @Override - public ShardSearchContextId getContextId() { - return original.getContextId(); - } - - @Override - public int getShardIndex() { - return original.getShardIndex(); - } - - @Override - public SearchShardTarget getSearchShardTarget() { - return original.getSearchShardTarget(); - } - - @Override - public void setSearchShardTarget(SearchShardTarget shardTarget) { - original.setSearchShardTarget(shardTarget); - } - - @Override - public void setShardIndex(int shardIndex) { - original.setShardIndex(shardIndex); - } - - @Override - public FetchSearchResult fetchResult() { - return original.fetchResult(); - } - - @Override - public ShardSearchRequest getShardSearchRequest() { - return original.getShardSearchRequest(); - } - - @Override - public void setShardSearchRequest(ShardSearchRequest shardSearchRequest) { - original.setShardSearchRequest(shardSearchRequest); - } - - @Override - public RescoreDocIds getRescoreDocIds() { - return original.getRescoreDocIds(); - } - - @Override - public boolean equals(Object obj) { - return original.equals(obj); - } - - @Override - public void setRescoreDocIds(RescoreDocIds rescoreDocIds) { - original.setRescoreDocIds(rescoreDocIds); - } - - @Override - public boolean isNull() { - return original.isNull(); - } - - @Override - public QuerySearchResult queryResult() { - return this; - } - - @Override - public void searchTimedOut(boolean searchTimedOut) { - original.searchTimedOut(searchTimedOut); - } - - @Override - public boolean searchTimedOut() { - return original.searchTimedOut(); - } - - @Override - public void terminatedEarly(boolean terminatedEarly) { - original.terminatedEarly(terminatedEarly); - } - - @Override - public Boolean terminatedEarly() { - return original.terminatedEarly(); - } - - @Override - public TopDocsAndMaxScore topDocs() { - return original.topDocs(); - } - - @Override - public boolean hasConsumedTopDocs() { - return original.hasConsumedTopDocs(); - } - - @Override - public TopDocsAndMaxScore consumeTopDocs() { - return original.consumeTopDocs(); - } - - @Override - public void topDocs(TopDocsAndMaxScore topDocs, DocValueFormat[] sortValueFormats) { - original.topDocs(topDocs, sortValueFormats); - } - - @Override - public DocValueFormat[] sortValueFormats() { - return original.sortValueFormats(); - } - - @Override - public boolean hasAggs() { - return original.hasAggs(); - } - - @Override - public InternalAggregations consumeAggs() { - return original.consumeAggs(); - } - - @Override - public void releaseAggs() { - original.releaseAggs(); - } - - @Override - public void aggregations(InternalAggregations aggregations) { - original.aggregations(aggregations); - } - - @Override - public DelayableWriteable aggregations() { - return original.aggregations(); - } - - @Override - public ProfileShardResult consumeProfileResult() { - return original.consumeProfileResult(); - } - - @Override - public boolean hasProfileResults() { - return original.hasProfileResults(); - } - - @Override - public void consumeAll() { - original.consumeAll(); - } - - @Override - public String toString() { - return original.toString(); - } - - @Override - public void profileResults(ProfileShardResult shardResults) { - original.profileResults(shardResults); - } - - @Override - public Suggest suggest() { - return original.suggest(); - } - - @Override - public void suggest(Suggest suggest) { - original.suggest(suggest); - } - - @Override - public int from() { - return original.from(); - } - - @Override - public QuerySearchResult from(int from) { - return original.from(from); - } - - @Override - public int size() { - return original.size(); - } - - @Override - public QuerySearchResult size(int size) { - return original.size(size); - } - - @Override - public long serviceTimeEWMA() { - return original.serviceTimeEWMA(); - } - - @Override - public QuerySearchResult serviceTimeEWMA(long serviceTimeEWMA) { - return original.serviceTimeEWMA(serviceTimeEWMA); - } - - @Override - public int nodeQueueSize() { - return original.nodeQueueSize(); - } - - @Override - public QuerySearchResult nodeQueueSize(int nodeQueueSize) { - return original.nodeQueueSize(nodeQueueSize); - } - - @Override - public boolean hasSuggestHits() { - return original.hasSuggestHits(); - } - - @Override - public boolean hasSearchContext() { - return original.hasSearchContext(); - } - - @Override - public void readFromWithId(ShardSearchContextId id, StreamInput in) throws IOException { - original.readFromWithId(id, in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - original.writeTo(out); - } - - @Override - public void writeToNoId(StreamOutput out) throws IOException { - original.writeToNoId(out); - } - - @Override - public TotalHits getTotalHits() { - return original.getTotalHits(); - } - - @Override - public float getMaxScore() { - return original.getMaxScore(); - } -} \ No newline at end of file From 3b82d1d035d00e6e3a6684039b50626e109f8a99 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 9 Sep 2021 13:54:55 +0200 Subject: [PATCH 22/31] Adapter only needs connection version --- .../elasticsearch/action/search/FetchSearchPhase.java | 2 +- .../action/search/FieldsOptionSourceAdapter.java | 6 ++---- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- .../action/search/TransportSearchAction.java | 10 +++++----- .../action/search/FieldsOptionSourceAdapterTests.java | 7 ++----- 5 files changed, 11 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index ae5726773d3d2..cd49848528648 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -165,7 +165,7 @@ private void executeFetch(final int shardIndex, final SearchShardTarget shardTar final Transport.Connection connection) { final FieldsOptionSourceAdapter adapter; if (context.getRequest().isFieldsOptionEmulationEnabled()) { - adapter = FieldsOptionSourceAdapter.create(connection, context.getRequest().source()); + adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), context.getRequest().source()); } else { adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; } diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java index ea146325c611b..75384a97206b9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -17,7 +17,6 @@ import org.elasticsearch.search.fetch.subphase.FieldAndFormat; import org.elasticsearch.search.fetch.subphase.FieldFetcher; import org.elasticsearch.search.lookup.SourceLookup; -import org.elasticsearch.transport.Transport.Connection; import java.io.IOException; import java.util.Collections; @@ -41,9 +40,8 @@ default boolean getRemoveSourceOnResponse() { String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + "a search request that is targeting pre version 7.10 nodes."; - static FieldsOptionSourceAdapter create(Connection connection, SearchSourceBuilder searchSource) { - Version version = connection.getVersion(); - if (version.before(Version.V_7_10_0)) { + static FieldsOptionSourceAdapter create(Version connectionVersion, SearchSourceBuilder searchSource) { + if (connectionVersion.before(Version.V_7_10_0)) { List fetchFields = searchSource.fetchFields(); if (fetchFields != null && fetchFields.isEmpty() == false) { String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index f74785293e4ef..f423b1f43a10c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -72,7 +72,7 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); if (getRequest().isFieldsOptionEmulationEnabled()) { - FieldsOptionSourceAdapter fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection, request.source()); + FieldsOptionSourceAdapter fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection.getVersion(), request.source()); fieldsOptionAdapter.adaptRequest(request::source); } getSearchTransport().sendExecuteQuery(connection, request, getTask(), 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 3fd33ab39b5f2..003123ce9ab1a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -63,6 +63,7 @@ import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.Transport.Connection; import org.elasticsearch.transport.TransportService; import java.util.ArrayList; @@ -365,7 +366,8 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; if (searchRequest.isFieldsOptionEmulationEnabled()) { try { - adapter = FieldsOptionSourceAdapter.create(remoteClusterService.getConnection(clusterAlias), searchRequest.source()); + Connection connection = remoteClusterService.getConnection(clusterAlias); + adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), searchRequest.source()); } catch (NoSuchRemoteClusterException ex) { // no connection version, adapter creation not possible if cluster not connected } @@ -415,10 +417,8 @@ public void onFailure(Exception e) { FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; if (searchRequest.isFieldsOptionEmulationEnabled()) { try { - adapter = FieldsOptionSourceAdapter.create( - remoteClusterService.getConnection(clusterAlias), - searchRequest.source() - ); + Connection connection = remoteClusterService.getConnection(clusterAlias); + adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), searchRequest.source()); } catch (NoSuchRemoteClusterException ex) { // don't create fields option adapter in this case } diff --git a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java index dcb091cb44d43..60df25e04d296 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java @@ -10,14 +10,12 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; -import org.elasticsearch.transport.Transport; public class FieldsOptionSourceAdapterTests extends ESTestCase { @@ -153,8 +151,7 @@ public void testFieldsAdapterException() { assertEquals(FieldsOptionSourceAdapter.FIELDS_EMULATION_ERROR_MSG, iae.getMessage()); } - private Transport.Connection mockConnection(Version version) { - DiscoveryNode node = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), version); - return new SearchAsyncActionTests.MockConnection(node); + private Version mockConnection(Version version) { + return version; } } From f9df7dd5cbb910297727a47fd75222c3060802aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 10 Sep 2021 12:27:56 +0200 Subject: [PATCH 23/31] Rework and simplify adapter class --- .../action/search/FetchSearchPhase.java | 16 +- .../search/FieldsOptionSourceAdapter.java | 175 +++++++++--------- .../SearchQueryThenFetchAsyncAction.java | 6 +- .../action/search/TransportSearchAction.java | 46 ++--- .../FieldsOptionSourceAdapterTests.java | 116 ++++++------ 5 files changed, 172 insertions(+), 187 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index cd49848528648..0d73aa4e37d59 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -123,6 +123,8 @@ private void innerRun() throws Exception { final CountedCollector counter = new CountedCollector<>(fetchResults, docIdsToLoad.length, // we count down every shard in the result no matter if we got any results or not finishPhase, context); + + FieldsOptionSourceAdapter fieldsOptionAdapter = new FieldsOptionSourceAdapter(context.getRequest()); for (int i = 0; i < docIdsToLoad.length; i++) { IntArrayList entry = docIdsToLoad[i]; SearchPhaseResult queryResult = queryResults.get(i); @@ -144,7 +146,7 @@ private void innerRun() throws Exception { lastEmittedDocPerShard, searchShardTarget.getOriginalIndices(), queryResult.getShardSearchRequest(), queryResult.getRescoreDocIds()); executeFetch(queryResult.getShardIndex(), searchShardTarget, counter, fetchSearchRequest, queryResult.queryResult(), - connection); + connection, fieldsOptionAdapter); } } } @@ -162,20 +164,14 @@ protected ShardFetchSearchRequest createFetchRequest(ShardSearchContextId contex private void executeFetch(final int shardIndex, final SearchShardTarget shardTarget, final CountedCollector counter, final ShardFetchSearchRequest fetchSearchRequest, final QuerySearchResult querySearchResult, - final Transport.Connection connection) { - final FieldsOptionSourceAdapter adapter; - if (context.getRequest().isFieldsOptionEmulationEnabled()) { - adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), context.getRequest().source()); - } else { - adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; - } - + final Transport.Connection connection, + final FieldsOptionSourceAdapter fieldsOptionAdapter) { context.getSearchTransport().sendExecuteFetch(connection, fetchSearchRequest, context.getTask(), new SearchActionListener(shardTarget, shardIndex) { @Override public void innerOnResponse(FetchSearchResult result) { try { - adapter.adaptResponse(result.hits().getHits()); + fieldsOptionAdapter.adaptResponse(connection.getVersion(), result.hits().getHits()); progressListener.notifyFetchResult(shardIndex); counter.onResult(result); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java index 75384a97206b9..61ac7dda6769e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -25,106 +25,109 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -interface FieldsOptionSourceAdapter { - default void adaptRequest(Consumer sourceConsumer) { - // noop - }; - default void adaptResponse(SearchHit[] searchHits) { - // noop - }; +public class FieldsOptionSourceAdapter { - default boolean getRemoveSourceOnResponse() { - return false; - } - - String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + static String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + "a search request that is targeting pre version 7.10 nodes."; - static FieldsOptionSourceAdapter create(Version connectionVersion, SearchSourceBuilder searchSource) { - if (connectionVersion.before(Version.V_7_10_0)) { - List fetchFields = searchSource.fetchFields(); - if (fetchFields != null && fetchFields.isEmpty() == false) { - String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); - CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; - // We separate the "include_unmapped" field patters with wildcards from the rest in order to use less space in the - // lookup automaton - Map> partitions = fetchFields.stream() - .map(ff -> ff.field) - .collect(Collectors.partitioningBy((s -> Regex.isSimpleMatchPattern(s)))); - List unmappedWildcardPattern = partitions.get(true); - List unmappedConcreteFields = partitions.get(false); - if (unmappedWildcardPattern.isEmpty() == false) { - unmappedFieldsFetchAutomaton = new CharacterRunAutomaton( - Regex.simpleMatchToAutomaton(unmappedWildcardPattern.toArray(new String[unmappedWildcardPattern.size()])), - 100000 - ); - } - final FieldFetcher fieldFetcher = new FieldFetcher( - Collections.emptyMap(), - unmappedFieldsFetchAutomaton, - unmappedConcreteFields - ); - - FetchSourceContext fetchSource = searchSource.fetchSource(); - final boolean removeSourceOnResponse; - final SearchSourceBuilder adaptedSource; + private final SearchSourceBuilder originalSource; + private final boolean requestShouldBeAdapted; + private final boolean removeSourceOnResponse; + private final SearchSourceBuilder adaptedSource; + private FieldFetcher fieldFetcher; - if (fetchSource == null || (fetchSource != null && fetchSource.fetchSource())) { - // case 1: original request has source: true, but no includes/exclude -> do nothing on request - if (fetchSource == null || fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { - // change nothing, we can get everything from source and can leave it when translating the response - removeSourceOnResponse = false; - adaptedSource = searchSource; - } else { - // original request has source includes/excludes set. In this case we don't want to silently - // overwrite the source parameter with something else, so we error instead - throw new IllegalArgumentException(FIELDS_EMULATION_ERROR_MSG); - } + public FieldsOptionSourceAdapter(SearchRequest request) { + originalSource = request.source() != null ? request.source() : new SearchSourceBuilder(); + List fetchFields = originalSource.fetchFields(); + requestShouldBeAdapted = request.isFieldsOptionEmulationEnabled() && fetchFields != null && fetchFields.isEmpty() == false; + if (requestShouldBeAdapted) { + FetchSourceContext fetchSource = originalSource.fetchSource(); + if (fetchSource == null || (fetchSource != null && fetchSource.fetchSource())) { + // case 1: original request has source: true, but no includes/exclude -> do nothing on request + if (fetchSource == null || fetchSource.includes().length == 0 && fetchSource.excludes().length == 0) { + // change nothing, we can get everything from source and can leave it when translating the response + removeSourceOnResponse = false; + adaptedSource = originalSource; } else { - // case 2: original request has source: false - adaptedSource = searchSource.shallowCopy(); - adaptedSource.fetchSource(new FetchSourceContext(true)); - adaptedSource.fetchSource(includes, null); - removeSourceOnResponse = true; + // original request has source includes/excludes set. In this case we don't want to silently + // overwrite the source parameter with something else, so we error instead + throw new IllegalArgumentException(FIELDS_EMULATION_ERROR_MSG); } + } else { + // case 2: original request has source: false + adaptedSource = originalSource.shallowCopy(); + adaptedSource.fetchSource(new FetchSourceContext(true)); + String[] includes = fetchFields.stream().map(ff -> ff.field).toArray(i -> new String[i]); + adaptedSource.fetchSource(includes, null); + removeSourceOnResponse = true; + } - return new FieldsOptionSourceAdapter() { + } else { + removeSourceOnResponse = false; + adaptedSource = null; + } + } - @Override - public void adaptRequest(Consumer sourceConsumer) { - sourceConsumer.accept(adaptedSource); - } + /** + * Swaps the existing search source with one that has "source" fetching enabled and configured so that + * we retrieve the fields requested by the "fields" option in the original request. + * This is only done for connections to pre-7.10 nodes and if the fields emulation has been enabled, otherwise + * calling this method will be a no-op. + */ + public void adaptRequest(Version connectionVersion, Consumer sourceConsumer) { + if (requestShouldBeAdapted && connectionVersion.before(Version.V_7_10_0) && adaptedSource != null) { + sourceConsumer.accept(adaptedSource); + } + } - @Override - public void adaptResponse(SearchHit[] hits) { - for (SearchHit hit : hits) { - SourceLookup lookup = new SourceLookup(); - lookup.setSource(hit.getSourceAsMap()); - Map documentFields = Collections.emptyMap(); - try { - documentFields = fieldFetcher.fetch(lookup); - } catch (IOException e) { - // best effort fetching field, if this doesn't work continue - } - for (Map.Entry entry : documentFields.entrySet()) { - hit.setDocumentField(entry.getKey(), entry.getValue()); - } - if (removeSourceOnResponse) { - // original request didn't request source, so we remove it - hit.sourceRef(null); - } + /** + * Goes through all hits in the response and fetches fields requested by the "fields" option in the original request by + * fetching them from source. If the original request has "source" disabled, this method will also delete the + * source section in the hit. + * This is only done for connections to pre-7.10 nodes and if the fields emulation has been enabled, otherwise + * calling this method will be a no-op. + */ + public void adaptResponse(Version connectionVersion, SearchHit[] hits) { + if (requestShouldBeAdapted && connectionVersion.before(Version.V_7_10_0)) { + for (SearchHit hit : hits) { + SourceLookup lookup = new SourceLookup(); + lookup.setSource(hit.getSourceAsMap()); + Map documentFields = Collections.emptyMap(); + try { + if (fieldFetcher == null) { + CharacterRunAutomaton unmappedFieldsFetchAutomaton = null; + // We separate the "include_unmapped" field patters with wildcards from the rest in order to use less space in the + // lookup automaton + Map> partitions = originalSource.fetchFields().stream() + .map(ff -> ff.field) + .collect(Collectors.partitioningBy((s -> Regex.isSimpleMatchPattern(s)))); + List unmappedWildcardPattern = partitions.get(true); + List unmappedConcreteFields = partitions.get(false); + if (unmappedWildcardPattern.isEmpty() == false) { + unmappedFieldsFetchAutomaton = new CharacterRunAutomaton( + Regex.simpleMatchToAutomaton(unmappedWildcardPattern.toArray(new String[unmappedWildcardPattern.size()])), + 100000 + ); } - } + fieldFetcher = new FieldFetcher(Collections.emptyMap(), unmappedFieldsFetchAutomaton, unmappedConcreteFields); - @Override - public boolean getRemoveSourceOnResponse() { - return removeSourceOnResponse; } - }; + documentFields = fieldFetcher.fetch(lookup); + } catch (IOException e) { + // best effort fetching field, if this doesn't work continue + } + for (Map.Entry entry : documentFields.entrySet()) { + hit.setDocumentField(entry.getKey(), entry.getValue()); + } + if (removeSourceOnResponse) { + // original request didn't request source, so we remove it + hit.sourceRef(null); + } } } - return FieldsOptionSourceAdapter.NOOP_ADAPTER; } - FieldsOptionSourceAdapter NOOP_ADAPTER = new FieldsOptionSourceAdapter() {}; + boolean getRemoveSourceOnResponse() { + return removeSourceOnResponse; + } } \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index f423b1f43a10c..0bead9f2ac5e8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -71,10 +71,8 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final SearchActionListener listener) { ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); Connection connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); - if (getRequest().isFieldsOptionEmulationEnabled()) { - FieldsOptionSourceAdapter fieldsOptionAdapter = FieldsOptionSourceAdapter.create(connection.getVersion(), request.source()); - fieldsOptionAdapter.adaptRequest(request::source); - } + FieldsOptionSourceAdapter fieldsOptionAdapter = new FieldsOptionSourceAdapter(getRequest()); + fieldsOptionAdapter.adaptRequest(connection.getVersion(), request::source); getSearchTransport().sendExecuteQuery(connection, request, getTask(), 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 003123ce9ab1a..75be887421677 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.search; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; @@ -58,12 +59,10 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.Transport.Connection; import org.elasticsearch.transport.TransportService; import java.util.ArrayList; @@ -363,25 +362,18 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; - if (searchRequest.isFieldsOptionEmulationEnabled()) { - try { - Connection connection = remoteClusterService.getConnection(clusterAlias); - adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), searchRequest.source()); - } catch (NoSuchRemoteClusterException ex) { - // no connection version, adapter creation not possible if cluster not connected - } - } + SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - adapter.adaptRequest(ccsSearchRequest::source); - final FieldsOptionSourceAdapter finalAdapter = adapter; + final FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); + final Version connectionVersion = remoteClusterService.getConnection(clusterAlias).getVersion(); + adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { - finalAdapter.adaptResponse(searchResponse.getHits().getHits()); + adapter.adaptResponse(connectionVersion, searchResponse.getHits().getHits()); Map profileResults = searchResponse.getProfileResults(); SearchProfileResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileResults(profileResults); @@ -410,19 +402,11 @@ public void onFailure(Exception e) { final AtomicReference exceptions = new AtomicReference<>(); int totalClusters = remoteIndices.size() + (localIndices == null ? 0 : 1); final CountDown countDown = new CountDown(totalClusters); + FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); for (Map.Entry entry : remoteIndices.entrySet()) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.NOOP_ADAPTER; - if (searchRequest.isFieldsOptionEmulationEnabled()) { - try { - Connection connection = remoteClusterService.getConnection(clusterAlias); - adapter = FieldsOptionSourceAdapter.create(connection.getVersion(), searchRequest.source()); - } catch (NoSuchRemoteClusterException ex) { - // don't create fields option adapter in this case - } - } SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( parentTaskId, searchRequest, @@ -431,7 +415,8 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - adapter.adaptRequest(ccsSearchRequest::source); + Version connectionVersion = remoteClusterService.getConnection(clusterAlias).getVersion(); + adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); ActionListener ccsListener = createCCSListener( clusterAlias, skipUnavailable, @@ -441,7 +426,8 @@ public void onFailure(Exception e) { searchResponseMerger, totalClusters, listener, - adapter + adapter, + connectionVersion ); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, ccsListener); @@ -456,7 +442,8 @@ public void onFailure(Exception e) { searchResponseMerger, totalClusters, listener, - FieldsOptionSourceAdapter.NOOP_ADAPTER + null, + null ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); @@ -522,12 +509,15 @@ private static ActionListener createCCSListener(String clusterAl AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, int totalClusters, ActionListener originalListener, - FieldsOptionSourceAdapter adapter) { + FieldsOptionSourceAdapter adapter, + Version connectionVersion) { return new CCSActionListener(clusterAlias, skipUnavailable, countDown, skippedClusters, exceptions, originalListener) { @Override void innerOnResponse(SearchResponse searchResponse) { - adapter.adaptResponse(searchResponse.getHits().getHits()); + if (adapter != null && connectionVersion != null) { + adapter.adaptResponse(connectionVersion, searchResponse.getHits().getHits()); + } searchResponseMerger.add(searchResponse); } diff --git a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java index 60df25e04d296..672b89ee1b8e8 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FieldsOptionSourceAdapterTests.java @@ -19,39 +19,20 @@ public class FieldsOptionSourceAdapterTests extends ESTestCase { - /** - * test that on versions after 7.10 or when no fields are fetched, the returned adapter does nothing (is the noop adapter) - */ - public void testNoopFieldsAdapterCreation() { - SearchSourceBuilder source = new SearchSourceBuilder(); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); - assertSame(FieldsOptionSourceAdapter.NOOP_ADAPTER, adapter); - - adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_7_10_0, Version.CURRENT)), - source - ); - // no set fields should also return a noop adapter - assertSame(FieldsOptionSourceAdapter.NOOP_ADAPTER, adapter); - } - /** * test that with set fields pattern, the adapter does something. * check correct request and response modifications */ public void testFieldsAdapterNoSource() { - // build adapter that gets all foo* fields SearchSourceBuilder source = new SearchSourceBuilder().fetchField("foo*"); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(source); + searchRequest.setFieldsOptionEmulationEnabled(true); + FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); SetOnce rewrittenSource = new SetOnce<>(); - adapter.adaptRequest(rewrittenSource::set); + Version connectionVersion = VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3); + adapter.adaptRequest(connectionVersion, rewrittenSource::set); assertNull(rewrittenSource.get().fetchSource()); assertFalse(adapter.getRemoveSourceOnResponse()); @@ -60,7 +41,7 @@ public void testFieldsAdapterNoSource() { assertNull(hit.getFields().get("foo")); assertNull(hit.getFields().get("foo.bar")); assertNull(hit.getFields().get("fuzz")); - adapter.adaptResponse(searchHits); + adapter.adaptResponse(connectionVersion, searchHits); assertEquals(1, hit.getFields().get("foo").getValues().size()); assertEquals("bar", hit.getFields().get("foo").getValues().get(0)); assertEquals(2, hit.getFields().get("foo.bar").getValues().size()); @@ -74,13 +55,10 @@ public void testFieldsAdapterNoSource() { * Test that if request source is set to "false", we still set the source on the adapted request and remove it on return */ public void testFieldsAdapterSourceFalse() { - SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(false).fetchField("foo*"); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); + FieldsOptionSourceAdapter adapter = createAdapter("foo*", false); SetOnce rewrittenSource = new SetOnce<>(); - adapter.adaptRequest(rewrittenSource::set); + Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3); + adapter.adaptRequest(version, rewrittenSource::set); assertNotNull(rewrittenSource.get().fetchSource()); assertTrue(rewrittenSource.get().fetchSource().fetchSource()); assertEquals(1, rewrittenSource.get().fetchSource().includes().length); @@ -88,25 +66,22 @@ public void testFieldsAdapterSourceFalse() { assertTrue(adapter.getRemoveSourceOnResponse()); SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"foo\" : \"bar\", \"fuzz\":2, \"foo.bar\":[4,5]}")); + assertTrue(hit.hasSource()); SearchHit[] searchHits = new SearchHit[] { hit }; - adapter.adaptResponse(searchHits); + adapter.adaptResponse(version, searchHits); assertFalse(hit.hasSource()); } public void testFieldsAdapterSourceTrue() { - SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(true).fetchField("foo*"); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); + FieldsOptionSourceAdapter adapter = createAdapter("foo*"); SetOnce rewrittenSource = new SetOnce<>(); - adapter.adaptRequest(rewrittenSource::set); - assertSame(source, rewrittenSource.get()); + Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3); + adapter.adaptRequest(version, rewrittenSource::set); assertFalse(adapter.getRemoveSourceOnResponse()); SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"foo\" : \"bar\", \"fuzz\":2, \"foo.bar\":[4,5]}")); SearchHit[] searchHits = new SearchHit[] { hit }; - adapter.adaptResponse(searchHits); + adapter.adaptResponse(version, searchHits); assertTrue(hit.hasSource()); } @@ -115,43 +90,66 @@ public void testFieldsAdapterSourceTrue() { * return this via the fields API, since our regular lookup there only returns flattened leaf values */ public void testFieldsAdapterObjects() { - SearchSourceBuilder source = new SearchSourceBuilder().fetchSource(false).fetchField("obj"); - FieldsOptionSourceAdapter adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); + FieldsOptionSourceAdapter adapter = createAdapter("obj"); SearchHit hit = new SearchHit(1).sourceRef(new BytesArray("{ \"obj\": { \"foo\" : \"value\"}}")); SearchHit[] searchHits = new SearchHit[] { hit }; - adapter.adaptResponse(searchHits); + Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3); + adapter.adaptResponse(version, searchHits); assertNull(hit.getFields().get("obj")); - source = new SearchSourceBuilder().fetchSource(false).fetchField("obj.*"); - adapter = FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - source - ); + adapter = createAdapter("obj.*", false); hit = new SearchHit(1).sourceRef(new BytesArray("{ \"obj\": { \"foo\" : \"value\"}}")); searchHits = new SearchHit[] { hit }; - adapter.adaptResponse(searchHits); + adapter.adaptResponse(version, searchHits); assertNull(hit.getFields().get("obj")); assertEquals(1, hit.getFields().get("obj.foo").getValues().size()); assertEquals("value", hit.getFields().get("obj.foo").getValues().get(0)); } + /** + * test that on connections after 7.10, we don't modify the original search source + */ + public void testAdaptMethodNoopBehaviour() { + SearchSourceBuilder source = new SearchSourceBuilder().fetchField("foo*").fetchSource(false); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(source); + searchRequest.setFieldsOptionEmulationEnabled(true); + FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); + + Version connectionVersion = VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3); + adapter.adaptRequest(connectionVersion, searchRequest::source); + assertNotSame("Search source on request shouldn't be same after adaption", source, searchRequest.source()); + + connectionVersion = VersionUtils.randomVersionBetween(random(), Version.V_7_10_0, Version.CURRENT); + searchRequest.source(source); + adapter.adaptRequest(connectionVersion, searchRequest::source); + assertSame("Search source on request should be same after adaption", source, searchRequest.source()); + } + /** * when _source is enabled with includes/excludes on the original request, we error */ public void testFieldsAdapterException() { - IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> FieldsOptionSourceAdapter.create( - mockConnection(VersionUtils.randomVersionBetween(random(), Version.V_6_8_18, Version.V_7_9_3)), - new SearchSourceBuilder().fetchField("foo*").fetchSource(new FetchSourceContext(true, new String[] { "bar" }, new String[0])) - )); + SearchSourceBuilder source = new SearchSourceBuilder().fetchField("foo*") + .fetchSource(new FetchSourceContext(true, new String[] { "bar" }, new String[0])); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(source); + searchRequest.setFieldsOptionEmulationEnabled(true); + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> new FieldsOptionSourceAdapter(searchRequest)); assertEquals(FieldsOptionSourceAdapter.FIELDS_EMULATION_ERROR_MSG, iae.getMessage()); } - private Version mockConnection(Version version) { - return version; + private FieldsOptionSourceAdapter createAdapter(String fetchFields) { + return createAdapter(fetchFields, true); + } + + private FieldsOptionSourceAdapter createAdapter(String fetchFields, boolean fetchSource) { + SearchSourceBuilder source = new SearchSourceBuilder().fetchField(fetchFields).fetchSource(fetchSource); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(source); + searchRequest.setFieldsOptionEmulationEnabled(true); + return new FieldsOptionSourceAdapter(searchRequest); } } From 61ee322239212607120035985b56358134283daa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 10 Sep 2021 14:31:04 +0200 Subject: [PATCH 24/31] Fix FetchSearchPhaseTests due to insufficient mocking --- .../action/search/MockSearchPhaseContext.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index 4ce68085b10e6..e70eacbc1f651 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -11,9 +11,11 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; -import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.internal.InternalSearchResponse; @@ -97,7 +99,9 @@ public void onShardFailure(int shardIndex, @Nullable SearchShardTarget shardTarg @Override public Transport.Connection getConnection(String clusterAlias, String nodeId) { - return null; // null is ok here for this test + return new SearchAsyncActionTests.MockConnection( + new DiscoveryNode("id", new TransportAddress(TransportAddress.META_ADDRESS, 9300), Version.CURRENT) + ); } @Override From a5f419e6b414bbf542b32e26bb302dd6bb5854ac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 10 Sep 2021 16:29:28 +0200 Subject: [PATCH 25/31] Fix tests --- .../action/search/TransportSearchAction.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 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 75be887421677..b6b87ea8431c3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -59,6 +59,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; @@ -349,6 +350,15 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { source.collapse().getInnerHits().isEmpty(); } + private static Version getConnectionVersionFieldsEmulation(RemoteClusterService remoteClusterService, String clusterAlias) { + try { + return remoteClusterService.getConnection(clusterAlias).getVersion(); + } catch (NoSuchRemoteClusterException ex) { + // if the cluster is not connected, use CURRENT so fields emulation will be skipped + return Version.CURRENT; + } + } + static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, @@ -366,7 +376,7 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); final FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); - final Version connectionVersion = remoteClusterService.getConnection(clusterAlias).getVersion(); + final Version connectionVersion = getConnectionVersionFieldsEmulation(remoteClusterService, clusterAlias); adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); @@ -415,7 +425,7 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - Version connectionVersion = remoteClusterService.getConnection(clusterAlias).getVersion(); + final Version connectionVersion = getConnectionVersionFieldsEmulation(remoteClusterService, clusterAlias); adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); ActionListener ccsListener = createCCSListener( clusterAlias, From 29f56dc073ad207b3452246f60ce31da7315ade6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 13 Sep 2021 15:10:24 +0200 Subject: [PATCH 26/31] Don't support ccs remote reduce for fields Option emulation --- .../action/search/TransportSearchAction.java | 38 ++++--------------- 1 file changed, 8 insertions(+), 30 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 b6b87ea8431c3..4ed6854e8dbc8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -8,7 +8,6 @@ package org.elasticsearch.action.search; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; @@ -59,7 +58,6 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteTransportException; @@ -346,25 +344,21 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { return false; } SearchSourceBuilder source = searchRequest.source(); + if (searchRequest.isFieldsOptionEmulationEnabled() && source.fetchFields() != null && source.fetchFields().isEmpty() == false) { + // we don't support minimize_roundstrip for cases where we might need fields API emulation + return false; + } return source == null || source.collapse() == null || source.collapse().getInnerHits() == null || source.collapse().getInnerHits().isEmpty(); } - private static Version getConnectionVersionFieldsEmulation(RemoteClusterService remoteClusterService, String clusterAlias) { - try { - return remoteClusterService.getConnection(clusterAlias).getVersion(); - } catch (NoSuchRemoteClusterException ex) { - // if the cluster is not connected, use CURRENT so fields emulation will be skipped - return Version.CURRENT; - } - } - static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, SearchTimeProvider timeProvider, InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, RemoteClusterService remoteClusterService, ThreadPool threadPool, ActionListener listener, BiConsumer> localSearchConsumer) { + assert(false); if (localIndices == null && remoteIndices.size() == 1) { //if we are searching against a single remote cluster, we simply forward the original search request to such cluster //and we directly perform final reduction in the remote cluster @@ -375,15 +369,11 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - final FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); - final Version connectionVersion = getConnectionVersionFieldsEmulation(remoteClusterService, clusterAlias); - adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { - adapter.adaptResponse(connectionVersion, searchResponse.getHits().getHits()); Map profileResults = searchResponse.getProfileResults(); SearchProfileResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileResults(profileResults); @@ -412,7 +402,6 @@ public void onFailure(Exception e) { final AtomicReference exceptions = new AtomicReference<>(); int totalClusters = remoteIndices.size() + (localIndices == null ? 0 : 1); final CountDown countDown = new CountDown(totalClusters); - FieldsOptionSourceAdapter adapter = new FieldsOptionSourceAdapter(searchRequest); for (Map.Entry entry : remoteIndices.entrySet()) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); @@ -425,8 +414,6 @@ public void onFailure(Exception e) { timeProvider.getAbsoluteStartMillis(), false ); - final Version connectionVersion = getConnectionVersionFieldsEmulation(remoteClusterService, clusterAlias); - adapter.adaptRequest(connectionVersion, ccsSearchRequest::source); ActionListener ccsListener = createCCSListener( clusterAlias, skipUnavailable, @@ -435,9 +422,7 @@ public void onFailure(Exception e) { exceptions, searchResponseMerger, totalClusters, - listener, - adapter, - connectionVersion + listener ); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, ccsListener); @@ -451,9 +436,7 @@ public void onFailure(Exception e) { exceptions, searchResponseMerger, totalClusters, - listener, - null, - null + listener ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); @@ -518,16 +501,11 @@ Map createFinalResponse() { private static ActionListener createCCSListener(String clusterAlias, boolean skipUnavailable, CountDown countDown, AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, int totalClusters, - ActionListener originalListener, - FieldsOptionSourceAdapter adapter, - Version connectionVersion) { + ActionListener originalListener) { return new CCSActionListener(clusterAlias, skipUnavailable, countDown, skippedClusters, exceptions, originalListener) { @Override void innerOnResponse(SearchResponse searchResponse) { - if (adapter != null && connectionVersion != null) { - adapter.adaptResponse(connectionVersion, searchResponse.getHits().getHits()); - } searchResponseMerger.add(searchResponse); } From c8fcdecc782fce7725075a1d3b11568fb8006e54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 13 Sep 2021 16:16:52 +0200 Subject: [PATCH 27/31] iter --- .../org/elasticsearch/action/search/TransportSearchAction.java | 2 -- 1 file changed, 2 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 4ed6854e8dbc8..d3bc2011726fe 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -357,8 +357,6 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, RemoteClusterService remoteClusterService, ThreadPool threadPool, ActionListener listener, BiConsumer> localSearchConsumer) { - - assert(false); if (localIndices == null && remoteIndices.size() == 1) { //if we are searching against a single remote cluster, we simply forward the original search request to such cluster //and we directly perform final reduction in the remote cluster From 12792728b000eb89fd06f479935ff5dd6573847f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 13 Sep 2021 16:46:26 +0200 Subject: [PATCH 28/31] Adress review comments --- rest-api-spec/src/main/resources/rest-api-spec/api/search.json | 2 +- .../elasticsearch/action/search/FieldsOptionSourceAdapter.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 3d844220ba7dd..d127b188c7736 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -252,7 +252,7 @@ "min_compatible_shard_node":{ "type":"string", "description":"The minimum compatible version that all shards involved in search should have for this request to be successful" - } + } }, "body":{ "description":"The search definition using the Query DSL" diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java index 61ac7dda6769e..fc0d46dc42c6f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -25,7 +25,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -public class FieldsOptionSourceAdapter { +class FieldsOptionSourceAdapter { static String FIELDS_EMULATION_ERROR_MSG = "Cannot specify both 'fields' and '_source' 'includes' or 'excludes' in" + "a search request that is targeting pre version 7.10 nodes."; From fda2fec2e06f40c56127e7a881926e22c729056b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 13 Sep 2021 16:58:20 +0200 Subject: [PATCH 29/31] checkstyle --- .../elasticsearch/action/search/FieldsOptionSourceAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java index fc0d46dc42c6f..129d1891058c8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java +++ b/server/src/main/java/org/elasticsearch/action/search/FieldsOptionSourceAdapter.java @@ -36,7 +36,7 @@ class FieldsOptionSourceAdapter { private final SearchSourceBuilder adaptedSource; private FieldFetcher fieldFetcher; - public FieldsOptionSourceAdapter(SearchRequest request) { + FieldsOptionSourceAdapter(SearchRequest request) { originalSource = request.source() != null ? request.source() : new SearchSourceBuilder(); List fetchFields = originalSource.fetchFields(); requestShouldBeAdapted = request.isFieldsOptionEmulationEnabled() && fetchFields != null && fetchFields.isEmpty() == false; From 969430959cab2c3b589b5e4379692213e058a857 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 13 Sep 2021 17:46:36 +0200 Subject: [PATCH 30/31] fix test --- .../elasticsearch/action/search/TransportSearchAction.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 d3bc2011726fe..01973e26a2e22 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -344,7 +344,10 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { return false; } SearchSourceBuilder source = searchRequest.source(); - if (searchRequest.isFieldsOptionEmulationEnabled() && source.fetchFields() != null && source.fetchFields().isEmpty() == false) { + if (searchRequest.isFieldsOptionEmulationEnabled() + && source != null + && source.fetchFields() != null + && source.fetchFields().isEmpty() == false) { // we don't support minimize_roundstrip for cases where we might need fields API emulation return false; } From ac34a44ee515bc7adee608a5ba7313b30c1e2dc4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 14 Sep 2021 13:07:37 +0200 Subject: [PATCH 31/31] tidying up --- .../action/search/TransportSearchAction.java | 39 ++++--------------- .../search/builder/SearchSourceBuilder.java | 8 ++-- .../search/query/QuerySearchResult.java | 14 +++---- 3 files changed, 18 insertions(+), 43 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 902ab705317f6..8f46a09577310 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -360,6 +360,7 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or InternalAggregation.ReduceContextBuilder aggReduceContextBuilder, RemoteClusterService remoteClusterService, ThreadPool threadPool, ActionListener listener, BiConsumer> localSearchConsumer) { + if (localIndices == null && remoteIndices.size() == 1) { //if we are searching against a single remote cluster, we simply forward the original search request to such cluster //and we directly perform final reduction in the remote cluster @@ -367,10 +368,8 @@ static void ccsRemoteReduce(TaskId parentTaskId, SearchRequest searchRequest, Or String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), clusterAlias, timeProvider.getAbsoluteStartMillis(), true); - Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override @@ -408,38 +407,16 @@ public void onFailure(Exception e) { String clusterAlias = entry.getKey(); boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); OriginalIndices indices = entry.getValue(); - SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest( - parentTaskId, - searchRequest, - indices.indices(), - clusterAlias, - timeProvider.getAbsoluteStartMillis(), - false - ); - ActionListener ccsListener = createCCSListener( - clusterAlias, - skipUnavailable, - countDown, - skippedClusters, - exceptions, - searchResponseMerger, - totalClusters, - listener - ); + SearchRequest ccsSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, indices.indices(), + clusterAlias, timeProvider.getAbsoluteStartMillis(), false); + ActionListener ccsListener = createCCSListener(clusterAlias, skipUnavailable, countDown, + skippedClusters, exceptions, searchResponseMerger, totalClusters, listener); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); remoteClusterClient.search(ccsSearchRequest, ccsListener); } if (localIndices != null) { - ActionListener ccsListener = createCCSListener( - RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, - false, - countDown, - skippedClusters, - exceptions, - searchResponseMerger, - totalClusters, - listener - ); + ActionListener ccsListener = createCCSListener(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, + false, countDown, skippedClusters, exceptions, searchResponseMerger, totalClusters, listener); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest(parentTaskId, searchRequest, localIndices.indices(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false); localSearchConsumer.accept(ccsLocalSearchRequest, ccsListener); @@ -479,10 +456,8 @@ static void collectSearchShards(IndicesOptions indicesOptions, String preference boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias); Client clusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); final String[] indices = entry.getValue().indices(); - ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices) .indicesOptions(indicesOptions).local(true).preference(preference).routing(routing); - clusterClient.admin().cluster().searchShards(searchShardsRequest, new CCSActionListener>( clusterAlias, skipUnavailable, responsesCountDown, skippedClusters, exceptions, listener) { diff --git a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index fe329c71715d0..42e649bcf3d86 100644 --- a/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -10,6 +10,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; +import org.elasticsearch.core.Booleans; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -17,16 +20,13 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; -import org.elasticsearch.common.xcontent.ParseField; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.Booleans; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.Rewriteable; diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 0d740035d9fbd..747226b67e257 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -8,6 +8,12 @@ package org.elasticsearch.search.query; +import static java.util.Collections.emptyList; +import static org.elasticsearch.common.lucene.Lucene.readTopDocs; +import static org.elasticsearch.common.lucene.Lucene.writeTopDocs; + +import java.io.IOException; + import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.TotalHits; import org.elasticsearch.Version; @@ -28,13 +34,7 @@ import org.elasticsearch.search.profile.SearchProfileQueryPhaseResult; import org.elasticsearch.search.suggest.Suggest; -import java.io.IOException; - -import static java.util.Collections.emptyList; -import static org.elasticsearch.common.lucene.Lucene.readTopDocs; -import static org.elasticsearch.common.lucene.Lucene.writeTopDocs; - -public class QuerySearchResult extends SearchPhaseResult { +public final class QuerySearchResult extends SearchPhaseResult { private int from; private int size;