Remove InternalSearchResponse as well as most usage of SearchResponseSections (#103583)

This indirection isn't really necessary now that its use in the high
level rest client went away. -> flattened the `SearchResponse` class to
make ref counting easier -> removed `InternalSearchResponse` whose
functionality was completely redundant now -> removed serialization and
most of the usage of `SearchResponseSections`, only keeping this for the
few search execution spots that currently use it and where removing it
won't get us anything and will be a huge changeset.

enables the last steps in #102030
This commit is contained in:
Armin Braun 2023-12-20 18:57:36 +01:00 committed by GitHub
parent 2cd816ebfd
commit 7fc1fcb2ee
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
76 changed files with 990 additions and 952 deletions

View file

@ -21,7 +21,6 @@ import org.elasticsearch.plugin.noop.NoopPlugin;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
@ -45,15 +44,13 @@ public class TransportNoopSearchAction extends HandledTransportAction<SearchRequ
protected void doExecute(Task task, SearchRequest request, ActionListener<SearchResponse> listener) { protected void doExecute(Task task, SearchRequest request, ActionListener<SearchResponse> listener) {
listener.onResponse( listener.onResponse(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f), InternalAggregations.EMPTY,
InternalAggregations.EMPTY, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,

View file

@ -15,7 +15,6 @@ import org.apache.lucene.tests.util.LuceneTestCase;
import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -330,16 +329,7 @@ public class DatabaseNodeServiceTests extends ESTestCase {
} }
SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(hits, null, null, false, null, null, 0, null, 1, 1, 0, 1L, null, null);
new SearchResponseSections(hits, null, null, false, null, null, 0),
null,
1,
1,
0,
1L,
null,
null
);
toRelease.add(searchResponse::decRef); toRelease.add(searchResponse::decRef);
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionFuture<SearchResponse> actionFuture = mock(ActionFuture.class); ActionFuture<SearchResponse> actionFuture = mock(ActionFuture.class);

View file

@ -11,7 +11,7 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentParser;
@ -38,11 +38,9 @@ public class MultiSearchTemplateResponseTests extends AbstractXContentTestCase<M
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = totalShards - successfulShards; int skippedShards = totalShards - successfulShards;
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
SearchResponse.Clusters clusters = randomClusters(); SearchResponse.Clusters clusters = randomClusters();
SearchTemplateResponse searchTemplateResponse = new SearchTemplateResponse(); SearchTemplateResponse searchTemplateResponse = new SearchTemplateResponse();
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -75,11 +73,9 @@ public class MultiSearchTemplateResponseTests extends AbstractXContentTestCase<M
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = totalShards - successfulShards; int skippedShards = totalShards - successfulShards;
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
SearchResponse.Clusters clusters = randomClusters(); SearchResponse.Clusters clusters = randomClusters();
SearchTemplateResponse searchTemplateResponse = new SearchTemplateResponse(); SearchTemplateResponse searchTemplateResponse = new SearchTemplateResponse();
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,

View file

@ -14,7 +14,7 @@ import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentBuilder;
@ -54,10 +54,8 @@ public class SearchTemplateResponseTests extends AbstractXContentTestCase<Search
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, totalShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
return new SearchResponse( return SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -161,17 +159,14 @@ public class SearchTemplateResponseTests extends AbstractXContentTestCase<Search
hit.score(2.0f); hit.score(2.0f);
SearchHit[] hits = new SearchHit[] { hit }; SearchHit[] hits = new SearchHit[] { hit };
InternalSearchResponse internalSearchResponse = new InternalSearchResponse( SearchResponse searchResponse = new SearchResponse(
new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f), new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f),
null, null,
null, null,
null,
false, false,
null, null,
1 null,
); 1,
SearchResponse searchResponse = new SearchResponse(
internalSearchResponse,
null, null,
0, 0,
0, 0,

View file

@ -66,7 +66,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.tasks.TaskManager;
@ -574,9 +573,14 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
new TotalHits(0, TotalHits.Relation.EQUAL_TO), new TotalHits(0, TotalHits.Relation.EQUAL_TO),
0 0
); );
InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalResponse, hits,
null,
null,
false,
false,
null,
1,
scrollId(), scrollId(),
5, 5,
4, 4,

View file

@ -30,7 +30,6 @@ import org.elasticsearch.index.reindex.ClientScrollableHitSource;
import org.elasticsearch.index.reindex.ScrollableHitSource; import org.elasticsearch.index.reindex.ScrollableHitSource;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
@ -169,9 +168,14 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
new TotalHits(0, TotalHits.Relation.EQUAL_TO), new TotalHits(0, TotalHits.Relation.EQUAL_TO),
0 0
); );
InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false, 1);
return new SearchResponse( return new SearchResponse(
internalResponse, hits,
null,
null,
false,
false,
null,
1,
randomSimpleString(random(), 1, 10), randomSimpleString(random(), 1, 10),
5, 5,
4, 4,

View file

@ -37,7 +37,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.ESRestTestCase;
import org.elasticsearch.test.rest.ObjectPath; import org.elasticsearch.test.rest.ObjectPath;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
@ -92,18 +91,15 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
TransportSearchAction.TYPE.name(), TransportSearchAction.TYPE.name(),
EsExecutors.DIRECT_EXECUTOR_SERVICE, EsExecutors.DIRECT_EXECUTOR_SERVICE,
SearchRequest::new, SearchRequest::new,
(request, channel, task) -> { (request, channel, task) -> channel.sendResponse(
InternalSearchResponse response = new InternalSearchResponse( new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN),
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
1 null,
); 1,
SearchResponse searchResponse = new SearchResponse(
response,
null, null,
1, 1,
1, 1,
@ -111,9 +107,8 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
100, 100,
ShardSearchFailure.EMPTY_ARRAY, ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY SearchResponse.Clusters.EMPTY
); )
channel.sendResponse(searchResponse); )
}
); );
newService.registerRequestHandler( newService.registerRequestHandler(
ClusterStateAction.NAME, ClusterStateAction.NAME,

View file

@ -29,12 +29,12 @@ import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables; import org.elasticsearch.core.Releasables;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchContextMissingException;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.PointInTimeBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.internal.ShardSearchRequest;
@ -212,7 +212,9 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
// total hits is null in the response if the tracking of total hits is disabled // total hits is null in the response if the tracking of total hits is disabled
boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED;
sendSearchResponse( sendSearchResponse(
withTotalHits ? InternalSearchResponse.EMPTY_WITH_TOTAL_HITS : InternalSearchResponse.EMPTY_WITHOUT_TOTAL_HITS, withTotalHits
? new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1)
: new SearchResponseSections(SearchHits.EMPTY_WITHOUT_TOTAL_HITS, null, null, false, null, null, 1),
new AtomicArray<>(0) new AtomicArray<>(0)
); );
return; return;
@ -655,7 +657,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
} }
private SearchResponse buildSearchResponse( private SearchResponse buildSearchResponse(
InternalSearchResponse internalSearchResponse, SearchResponseSections internalSearchResponse,
ShardSearchFailure[] failures, ShardSearchFailure[] failures,
String scrollId, String scrollId,
String searchContextId String searchContextId
@ -682,7 +684,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
} }
@Override @Override
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) { public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) {
ShardSearchFailure[] failures = buildShardFailures(); ShardSearchFailure[] failures = buildShardFailures();
Boolean allowPartialResults = request.allowPartialSearchResults(); Boolean allowPartialResults = request.allowPartialSearchResults();
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";

View file

@ -18,7 +18,6 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.collapse.CollapseBuilder; import org.elasticsearch.search.collapse.CollapseBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -31,13 +30,13 @@ import java.util.function.Supplier;
*/ */
final class ExpandSearchPhase extends SearchPhase { final class ExpandSearchPhase extends SearchPhase {
private final SearchPhaseContext context; private final SearchPhaseContext context;
private final InternalSearchResponse searchResponse; private final SearchHits searchHits;
private final Supplier<SearchPhase> nextPhase; private final Supplier<SearchPhase> nextPhase;
ExpandSearchPhase(SearchPhaseContext context, InternalSearchResponse searchResponse, Supplier<SearchPhase> nextPhase) { ExpandSearchPhase(SearchPhaseContext context, SearchHits searchHits, Supplier<SearchPhase> nextPhase) {
super("expand"); super("expand");
this.context = context; this.context = context;
this.searchResponse = searchResponse; this.searchHits = searchHits;
this.nextPhase = nextPhase; this.nextPhase = nextPhase;
} }
@ -53,7 +52,7 @@ final class ExpandSearchPhase extends SearchPhase {
@Override @Override
public void run() { public void run() {
if (isCollapseRequest() && searchResponse.hits().getHits().length > 0) { if (isCollapseRequest() && searchHits.getHits().length > 0) {
SearchRequest searchRequest = context.getRequest(); SearchRequest searchRequest = context.getRequest();
CollapseBuilder collapseBuilder = searchRequest.source().collapse(); CollapseBuilder collapseBuilder = searchRequest.source().collapse();
final List<InnerHitBuilder> innerHitBuilders = collapseBuilder.getInnerHits(); final List<InnerHitBuilder> innerHitBuilders = collapseBuilder.getInnerHits();
@ -61,7 +60,7 @@ final class ExpandSearchPhase extends SearchPhase {
if (collapseBuilder.getMaxConcurrentGroupRequests() > 0) { if (collapseBuilder.getMaxConcurrentGroupRequests() > 0) {
multiRequest.maxConcurrentSearchRequests(collapseBuilder.getMaxConcurrentGroupRequests()); multiRequest.maxConcurrentSearchRequests(collapseBuilder.getMaxConcurrentGroupRequests());
} }
for (SearchHit hit : searchResponse.hits().getHits()) { for (SearchHit hit : searchHits.getHits()) {
BoolQueryBuilder groupQuery = new BoolQueryBuilder(); BoolQueryBuilder groupQuery = new BoolQueryBuilder();
Object collapseValue = hit.field(collapseBuilder.getField()).getValue(); Object collapseValue = hit.field(collapseBuilder.getField()).getValue();
if (collapseValue != null) { if (collapseValue != null) {
@ -85,7 +84,7 @@ final class ExpandSearchPhase extends SearchPhase {
} }
context.getSearchTransport().sendExecuteMultiSearch(multiRequest, context.getTask(), ActionListener.wrap(response -> { context.getSearchTransport().sendExecuteMultiSearch(multiRequest, context.getTask(), ActionListener.wrap(response -> {
Iterator<MultiSearchResponse.Item> it = response.iterator(); Iterator<MultiSearchResponse.Item> it = response.iterator();
for (SearchHit hit : searchResponse.hits.getHits()) { for (SearchHit hit : searchHits.getHits()) {
for (InnerHitBuilder innerHitBuilder : innerHitBuilders) { for (InnerHitBuilder innerHitBuilder : innerHitBuilders) {
MultiSearchResponse.Item item = it.next(); MultiSearchResponse.Item item = it.next();
if (item.isFailure()) { if (item.isFailure()) {

View file

@ -15,9 +15,9 @@ import org.elasticsearch.common.document.DocumentField;
import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.fetch.subphase.LookupField; import org.elasticsearch.search.fetch.subphase.LookupField;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteClusterService;
import java.util.ArrayList; import java.util.ArrayList;
@ -33,10 +33,10 @@ import java.util.stream.Collectors;
*/ */
final class FetchLookupFieldsPhase extends SearchPhase { final class FetchLookupFieldsPhase extends SearchPhase {
private final SearchPhaseContext context; private final SearchPhaseContext context;
private final InternalSearchResponse searchResponse; private final SearchResponseSections searchResponse;
private final AtomicArray<SearchPhaseResult> queryResults; private final AtomicArray<SearchPhaseResult> queryResults;
FetchLookupFieldsPhase(SearchPhaseContext context, InternalSearchResponse searchResponse, AtomicArray<SearchPhaseResult> queryResults) { FetchLookupFieldsPhase(SearchPhaseContext context, SearchResponseSections searchResponse, AtomicArray<SearchPhaseResult> queryResults) {
super("fetch_lookup_fields"); super("fetch_lookup_fields");
this.context = context; this.context = context;
this.searchResponse = searchResponse; this.searchResponse = searchResponse;
@ -47,9 +47,9 @@ final class FetchLookupFieldsPhase extends SearchPhase {
} }
private static List<Cluster> groupLookupFieldsByClusterAlias(InternalSearchResponse response) { private static List<Cluster> groupLookupFieldsByClusterAlias(SearchHits searchHits) {
final Map<String, List<SearchHit>> perClusters = new HashMap<>(); final Map<String, List<SearchHit>> perClusters = new HashMap<>();
for (SearchHit hit : response.hits.getHits()) { for (SearchHit hit : searchHits.getHits()) {
String clusterAlias = hit.getClusterAlias() != null ? hit.getClusterAlias() : RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY; String clusterAlias = hit.getClusterAlias() != null ? hit.getClusterAlias() : RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY;
if (hit.hasLookupFields()) { if (hit.hasLookupFields()) {
perClusters.computeIfAbsent(clusterAlias, k -> new ArrayList<>()).add(hit); perClusters.computeIfAbsent(clusterAlias, k -> new ArrayList<>()).add(hit);
@ -70,7 +70,7 @@ final class FetchLookupFieldsPhase extends SearchPhase {
@Override @Override
public void run() { public void run() {
final List<Cluster> clusters = groupLookupFieldsByClusterAlias(searchResponse); final List<Cluster> clusters = groupLookupFieldsByClusterAlias(searchResponse.hits);
if (clusters.isEmpty()) { if (clusters.isEmpty()) {
context.sendSearchResponse(searchResponse, queryResults); context.sendSearchResponse(searchResponse, queryResults);
return; return;

View file

@ -16,7 +16,6 @@ import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.dfs.AggregatedDfs;
import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.ShardFetchSearchRequest; import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
@ -31,7 +30,7 @@ import java.util.function.BiFunction;
final class FetchSearchPhase extends SearchPhase { final class FetchSearchPhase extends SearchPhase {
private final ArraySearchPhaseResults<FetchSearchResult> fetchResults; private final ArraySearchPhaseResults<FetchSearchResult> fetchResults;
private final AtomicArray<SearchPhaseResult> queryResults; private final AtomicArray<SearchPhaseResult> queryResults;
private final BiFunction<InternalSearchResponse, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory; private final BiFunction<SearchResponseSections, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory;
private final SearchPhaseContext context; private final SearchPhaseContext context;
private final Logger logger; private final Logger logger;
private final SearchPhaseResults<SearchPhaseResult> resultConsumer; private final SearchPhaseResults<SearchPhaseResult> resultConsumer;
@ -45,7 +44,7 @@ final class FetchSearchPhase extends SearchPhase {
context, context,
(response, queryPhaseResults) -> new ExpandSearchPhase( (response, queryPhaseResults) -> new ExpandSearchPhase(
context, context,
response, response.hits,
() -> new FetchLookupFieldsPhase(context, response, queryPhaseResults) () -> new FetchLookupFieldsPhase(context, response, queryPhaseResults)
) )
); );
@ -55,7 +54,7 @@ final class FetchSearchPhase extends SearchPhase {
SearchPhaseResults<SearchPhaseResult> resultConsumer, SearchPhaseResults<SearchPhaseResult> resultConsumer,
AggregatedDfs aggregatedDfs, AggregatedDfs aggregatedDfs,
SearchPhaseContext context, SearchPhaseContext context,
BiFunction<InternalSearchResponse, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory BiFunction<SearchResponseSections, AtomicArray<SearchPhaseResult>, SearchPhase> nextPhaseFactory
) { ) {
super("fetch"); super("fetch");
if (context.getNumShards() != resultConsumer.getNumShards()) { if (context.getNumShards() != resultConsumer.getNumShards()) {
@ -230,11 +229,12 @@ final class FetchSearchPhase extends SearchPhase {
SearchPhaseController.ReducedQueryPhase reducedQueryPhase, SearchPhaseController.ReducedQueryPhase reducedQueryPhase,
AtomicArray<? extends SearchPhaseResult> fetchResultsArr AtomicArray<? extends SearchPhaseResult> fetchResultsArr
) { ) {
final InternalSearchResponse internalResponse = SearchPhaseController.merge( context.executeNextPhase(
context.getRequest().scroll() != null, this,
reducedQueryPhase, nextPhaseFactory.apply(
fetchResultsArr SearchPhaseController.merge(context.getRequest().scroll() != null, reducedQueryPhase, fetchResultsArr),
queryResults
)
); );
context.executeNextPhase(this, nextPhaseFactory.apply(internalResponse, queryResults));
} }
} }

View file

@ -14,7 +14,6 @@ import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasable;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
@ -64,7 +63,7 @@ interface SearchPhaseContext extends Executor {
* @param internalSearchResponse the internal search response * @param internalSearchResponse the internal search response
* @param queryResults the results of the query phase * @param queryResults the results of the query phase
*/ */
void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults); void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults);
/** /**
* Notifies the top-level listener of the provided exception * Notifies the top-level listener of the provided exception

View file

@ -40,7 +40,6 @@ import org.elasticsearch.search.dfs.AggregatedDfs;
import org.elasticsearch.search.dfs.DfsKnnResults; import org.elasticsearch.search.dfs.DfsKnnResults;
import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.profile.SearchProfileQueryPhaseResult; import org.elasticsearch.search.profile.SearchProfileQueryPhaseResult;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
@ -355,13 +354,13 @@ public final class SearchPhaseController {
* Expects sortedDocs to have top search docs across all shards, optionally followed by top suggest docs for each named * Expects sortedDocs to have top search docs across all shards, optionally followed by top suggest docs for each named
* completion suggestion ordered by suggestion name * completion suggestion ordered by suggestion name
*/ */
public static InternalSearchResponse merge( public static SearchResponseSections merge(
boolean ignoreFrom, boolean ignoreFrom,
ReducedQueryPhase reducedQueryPhase, ReducedQueryPhase reducedQueryPhase,
AtomicArray<? extends SearchPhaseResult> fetchResultsArray AtomicArray<? extends SearchPhaseResult> fetchResultsArray
) { ) {
if (reducedQueryPhase.isEmptyResult) { if (reducedQueryPhase.isEmptyResult) {
return InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; return new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1);
} }
ScoreDoc[] sortedDocs = reducedQueryPhase.sortedTopDocs.scoreDocs; ScoreDoc[] sortedDocs = reducedQueryPhase.sortedTopDocs.scoreDocs;
var fetchResults = fetchResultsArray.asList(); var fetchResults = fetchResultsArray.asList();
@ -753,14 +752,14 @@ public final class SearchPhaseController {
* Creates a new search response from the given merged hits. * Creates a new search response from the given merged hits.
* @see #merge(boolean, ReducedQueryPhase, AtomicArray) * @see #merge(boolean, ReducedQueryPhase, AtomicArray)
*/ */
public InternalSearchResponse buildResponse(SearchHits hits, Collection<? extends SearchPhaseResult> fetchResults) { public SearchResponseSections buildResponse(SearchHits hits, Collection<? extends SearchPhaseResult> fetchResults) {
return new InternalSearchResponse( return new SearchResponseSections(
hits, hits,
aggregations, aggregations,
suggest, suggest,
buildSearchProfileResults(fetchResults),
timedOut, timedOut,
terminatedEarly, terminatedEarly,
buildSearchProfileResults(fetchResults),
numReducePhases numReducePhases
); );
} }

View file

@ -28,7 +28,6 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResult;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
@ -67,7 +66,13 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
private static final ParseField TERMINATED_EARLY = new ParseField("terminated_early"); private static final ParseField TERMINATED_EARLY = new ParseField("terminated_early");
private static final ParseField NUM_REDUCE_PHASES = new ParseField("num_reduce_phases"); private static final ParseField NUM_REDUCE_PHASES = new ParseField("num_reduce_phases");
private final SearchResponseSections internalResponse; private final SearchHits hits;
private final Aggregations aggregations;
private final Suggest suggest;
private final SearchProfileResults profileResults;
private final boolean timedOut;
private final Boolean terminatedEarly;
private final int numReducePhases;
private final String scrollId; private final String scrollId;
private final String pointInTimeId; private final String pointInTimeId;
private final int totalShards; private final int totalShards;
@ -79,7 +84,13 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
public SearchResponse(StreamInput in) throws IOException { public SearchResponse(StreamInput in) throws IOException {
super(in); super(in);
internalResponse = new InternalSearchResponse(in); this.hits = new SearchHits(in);
this.aggregations = in.readBoolean() ? InternalAggregations.readFrom(in) : null;
this.suggest = in.readBoolean() ? new Suggest(in) : null;
this.timedOut = in.readBoolean();
this.terminatedEarly = in.readOptionalBoolean();
this.profileResults = in.readOptionalWriteable(SearchProfileResults::new);
this.numReducePhases = in.readVInt();
totalShards = in.readVInt(); totalShards = in.readVInt();
successfulShards = in.readVInt(); successfulShards = in.readVInt();
int size = in.readVInt(); int size = in.readVInt();
@ -99,7 +110,13 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
} }
public SearchResponse( public SearchResponse(
SearchResponseSections internalResponse, SearchHits hits,
Aggregations aggregations,
Suggest suggest,
boolean timedOut,
Boolean terminatedEarly,
SearchProfileResults profileResults,
int numReducePhases,
String scrollId, String scrollId,
int totalShards, int totalShards,
int successfulShards, int successfulShards,
@ -108,11 +125,27 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
ShardSearchFailure[] shardFailures, ShardSearchFailure[] shardFailures,
Clusters clusters Clusters clusters
) { ) {
this(internalResponse, scrollId, totalShards, successfulShards, skippedShards, tookInMillis, shardFailures, clusters, null); this(
hits,
aggregations,
suggest,
timedOut,
terminatedEarly,
profileResults,
numReducePhases,
scrollId,
totalShards,
successfulShards,
skippedShards,
tookInMillis,
shardFailures,
clusters,
null
);
} }
public SearchResponse( public SearchResponse(
SearchResponseSections internalResponse, SearchResponseSections searchResponseSections,
String scrollId, String scrollId,
int totalShards, int totalShards,
int successfulShards, int successfulShards,
@ -122,7 +155,49 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
Clusters clusters, Clusters clusters,
String pointInTimeId String pointInTimeId
) { ) {
this.internalResponse = internalResponse; this(
searchResponseSections.hits,
searchResponseSections.aggregations,
searchResponseSections.suggest,
searchResponseSections.timedOut,
searchResponseSections.terminatedEarly,
searchResponseSections.profileResults,
searchResponseSections.numReducePhases,
scrollId,
totalShards,
successfulShards,
skippedShards,
tookInMillis,
shardFailures,
clusters,
pointInTimeId
);
}
public SearchResponse(
SearchHits hits,
Aggregations aggregations,
Suggest suggest,
boolean timedOut,
Boolean terminatedEarly,
SearchProfileResults profileResults,
int numReducePhases,
String scrollId,
int totalShards,
int successfulShards,
int skippedShards,
long tookInMillis,
ShardSearchFailure[] shardFailures,
Clusters clusters,
String pointInTimeId
) {
this.hits = hits;
this.aggregations = aggregations;
this.suggest = suggest;
this.profileResults = profileResults;
this.timedOut = timedOut;
this.terminatedEarly = terminatedEarly;
this.numReducePhases = numReducePhases;
this.scrollId = scrollId; this.scrollId = scrollId;
this.pointInTimeId = pointInTimeId; this.pointInTimeId = pointInTimeId;
this.clusters = clusters; this.clusters = clusters;
@ -144,7 +219,7 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
* The search hits. * The search hits.
*/ */
public SearchHits getHits() { public SearchHits getHits() {
return internalResponse.hits(); return hits;
} }
/** /**
@ -152,7 +227,7 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
* either {@code null} or {@link InternalAggregations#EMPTY}. * either {@code null} or {@link InternalAggregations#EMPTY}.
*/ */
public @Nullable Aggregations getAggregations() { public @Nullable Aggregations getAggregations() {
return internalResponse.aggregations(); return aggregations;
} }
/** /**
@ -163,14 +238,14 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
} }
public Suggest getSuggest() { public Suggest getSuggest() {
return internalResponse.suggest(); return suggest;
} }
/** /**
* Has the search operation timed out. * Has the search operation timed out.
*/ */
public boolean isTimedOut() { public boolean isTimedOut() {
return internalResponse.timedOut(); return timedOut;
} }
/** /**
@ -178,14 +253,14 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
* <code>terminateAfter</code> * <code>terminateAfter</code>
*/ */
public Boolean isTerminatedEarly() { public Boolean isTerminatedEarly() {
return internalResponse.terminatedEarly(); return terminatedEarly;
} }
/** /**
* Returns the number of reduce phases applied to obtain this search response * Returns the number of reduce phases applied to obtain this search response
*/ */
public int getNumReducePhases() { public int getNumReducePhases() {
return internalResponse.getNumReducePhases(); return numReducePhases;
} }
/** /**
@ -253,7 +328,10 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
*/ */
@Nullable @Nullable
public Map<String, SearchProfileShardResult> getProfileResults() { public Map<String, SearchProfileShardResult> getProfileResults() {
return internalResponse.profile(); if (profileResults == null) {
return Collections.emptyMap();
}
return profileResults.getShardResults();
} }
/** /**
@ -278,7 +356,27 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
return Iterators.concat( return Iterators.concat(
ChunkedToXContentHelper.singleChunk(SearchResponse.this::headerToXContent), ChunkedToXContentHelper.singleChunk(SearchResponse.this::headerToXContent),
Iterators.single(clusters), Iterators.single(clusters),
internalResponse.toXContentChunked(params) Iterators.concat(
Iterators.flatMap(Iterators.single(hits), r -> r.toXContentChunked(params)),
Iterators.single((ToXContent) (b, p) -> {
if (aggregations != null) {
aggregations.toXContent(b, p);
}
return b;
}),
Iterators.single((b, p) -> {
if (suggest != null) {
suggest.toXContent(b, p);
}
return b;
}),
Iterators.single((b, p) -> {
if (profileResults != null) {
profileResults.toXContent(b, p);
}
return b;
})
)
); );
} }
@ -396,17 +494,14 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
} }
} }
} }
SearchResponseSections searchResponseSections = new SearchResponseSections( return new SearchResponse(
hits, hits,
aggs, aggs,
suggest, suggest,
timedOut, timedOut,
terminatedEarly, terminatedEarly,
profile, profile,
numReducePhases numReducePhases,
);
return new SearchResponse(
searchResponseSections,
scrollId, scrollId,
totalShards, totalShards,
successfulShards, successfulShards,
@ -420,7 +515,13 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
internalResponse.writeTo(out); hits.writeTo(out);
out.writeOptionalWriteable((InternalAggregations) aggregations);
out.writeOptionalWriteable(suggest);
out.writeBoolean(timedOut);
out.writeOptionalBoolean(terminatedEarly);
out.writeOptionalWriteable(profileResults);
out.writeVInt(numReducePhases);
out.writeVInt(totalShards); out.writeVInt(totalShards);
out.writeVInt(successfulShards); out.writeVInt(successfulShards);
@ -1268,17 +1369,14 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
// public for tests // public for tests
public static SearchResponse empty(Supplier<Long> tookInMillisSupplier, Clusters clusters) { public static SearchResponse empty(Supplier<Long> tookInMillisSupplier, Clusters clusters) {
SearchHits searchHits = new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), Float.NaN); SearchHits searchHits = new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), Float.NaN);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse( return new SearchResponse(
searchHits, searchHits,
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
0 null,
); 0,
return new SearchResponse(
internalSearchResponse,
null, null,
0, 0,
0, 0,

View file

@ -27,7 +27,6 @@ import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.AggregationReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResult;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
@ -211,18 +210,15 @@ final class SearchResponseMerger implements Releasable {
SearchProfileResults profileShardResults = profileResults.isEmpty() ? null : new SearchProfileResults(profileResults); SearchProfileResults profileShardResults = profileResults.isEmpty() ? null : new SearchProfileResults(profileResults);
// make failures ordering consistent between ordinary search and CCS by looking at the shard they come from // make failures ordering consistent between ordinary search and CCS by looking at the shard they come from
Arrays.sort(shardFailures, FAILURES_COMPARATOR); Arrays.sort(shardFailures, FAILURES_COMPARATOR);
InternalSearchResponse response = new InternalSearchResponse( long tookInMillis = searchTimeProvider.buildTookInMillis();
return new SearchResponse(
mergedSearchHits, mergedSearchHits,
reducedAggs, reducedAggs,
suggest, suggest,
profileShardResults,
topDocsStats.timedOut, topDocsStats.timedOut,
topDocsStats.terminatedEarly, topDocsStats.terminatedEarly,
numReducePhases profileShardResults,
); numReducePhases,
long tookInMillis = searchTimeProvider.buildTookInMillis();
return new SearchResponse(
response,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,

View file

@ -8,30 +8,20 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ChunkedToXContent;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResult;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.xcontent.ToXContent;
import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Iterator;
import java.util.Map; import java.util.Map;
/** /**
* Base class that holds the various sections which a search response is * Holds some sections that a search response is composed of (hits, aggs, suggestions etc.) during some steps of the search response
* composed of (hits, aggs, suggestions etc.) and allows to retrieve them. * building.
*
* The reason why this class exists is that the high level REST client uses its own classes
* to parse aggregations into, which are not serializable. This is the common part that can be
* shared between core and client.
*/ */
public class SearchResponseSections implements ChunkedToXContent { public class SearchResponseSections {
protected final SearchHits hits; protected final SearchHits hits;
protected final Aggregations aggregations; protected final Aggregations aggregations;
@ -98,33 +88,4 @@ public class SearchResponseSections implements ChunkedToXContent {
} }
return profileResults.getShardResults(); return profileResults.getShardResults();
} }
@Override
public Iterator<? extends ToXContent> toXContentChunked(ToXContent.Params params) {
return Iterators.concat(
Iterators.flatMap(Iterators.single(hits), r -> r.toXContentChunked(params)),
Iterators.single((ToXContent) (b, p) -> {
if (aggregations != null) {
aggregations.toXContent(b, p);
}
return b;
}),
Iterators.single((b, p) -> {
if (suggest != null) {
suggest.toXContent(b, p);
}
return b;
}),
Iterators.single((b, p) -> {
if (profileResults != null) {
profileResults.toXContent(b, p);
}
return b;
})
);
}
protected void writeTo(StreamOutput out) throws IOException {
throw new UnsupportedOperationException();
}
} }

View file

@ -18,7 +18,6 @@ import org.elasticsearch.core.Nullable;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalScrollSearchRequest; import org.elasticsearch.search.internal.InternalScrollSearchRequest;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteClusterService;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
@ -240,7 +239,6 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
final AtomicArray<? extends SearchPhaseResult> fetchResults final AtomicArray<? extends SearchPhaseResult> fetchResults
) { ) {
try { try {
final InternalSearchResponse internalResponse = SearchPhaseController.merge(true, queryPhase, fetchResults);
// the scroll ID never changes we always return the same ID. This ID contains all the shards and their context ids // the scroll ID never changes we always return the same ID. This ID contains all the shards and their context ids
// such that we can talk to them again in the next roundtrip. // such that we can talk to them again in the next roundtrip.
String scrollId = null; String scrollId = null;
@ -250,7 +248,7 @@ abstract class SearchScrollAsyncAction<T extends SearchPhaseResult> implements R
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse( new SearchResponse(
internalResponse, SearchPhaseController.merge(true, queryPhase, fetchResults),
scrollId, scrollId,
this.scrollId.getContext().length, this.scrollId.getContext().length,
successfulOps.get(), successfulOps.get(),

View file

@ -28,12 +28,12 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -252,7 +252,10 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction<OpenP
@Override @Override
protected void doRun() { protected void doRun() {
sendSearchResponse(InternalSearchResponse.EMPTY_WITH_TOTAL_HITS, results.getAtomicArray()); sendSearchResponse(
new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1),
results.getAtomicArray()
);
} }
@Override @Override

View file

@ -63,10 +63,8 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.AggregationReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResult;
@ -542,19 +540,16 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
? null ? null
: new SearchProfileResults(profileResults); : new SearchProfileResults(profileResults);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(
searchResponse.getHits(),
(InternalAggregations) searchResponse.getAggregations(),
searchResponse.getSuggest(),
profile,
searchResponse.isTimedOut(),
searchResponse.isTerminatedEarly(),
searchResponse.getNumReducePhases()
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse( new SearchResponse(
internalSearchResponse, searchResponse.getHits(),
searchResponse.getAggregations(),
searchResponse.getSuggest(),
searchResponse.isTimedOut(),
searchResponse.isTerminatedEarly(),
profile,
searchResponse.getNumReducePhases(),
searchResponse.getScrollId(), searchResponse.getScrollId(),
searchResponse.getTotalShards(), searchResponse.getTotalShards(),
searchResponse.getSuccessfulShards(), searchResponse.getSuccessfulShards(),

View file

@ -1,80 +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.search.internal;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest;
import java.io.IOException;
/**
* {@link SearchResponseSections} subclass that can be serialized over the wire.
*/
public class InternalSearchResponse extends SearchResponseSections implements Writeable {
public static final InternalSearchResponse EMPTY_WITH_TOTAL_HITS = new InternalSearchResponse(
SearchHits.EMPTY_WITH_TOTAL_HITS,
null,
null,
null,
false,
null,
1
);
public static final InternalSearchResponse EMPTY_WITHOUT_TOTAL_HITS = new InternalSearchResponse(
SearchHits.EMPTY_WITHOUT_TOTAL_HITS,
null,
null,
null,
false,
null,
1
);
public InternalSearchResponse(
SearchHits hits,
InternalAggregations aggregations,
Suggest suggest,
SearchProfileResults profileResults,
boolean timedOut,
Boolean terminatedEarly,
int numReducePhases
) {
super(hits, aggregations, suggest, timedOut, terminatedEarly, profileResults, numReducePhases);
}
public InternalSearchResponse(StreamInput in) throws IOException {
super(
new SearchHits(in),
in.readBoolean() ? InternalAggregations.readFrom(in) : null,
in.readBoolean() ? new Suggest(in) : null,
in.readBoolean(),
in.readOptionalBoolean(),
in.readOptionalWriteable(SearchProfileResults::new),
in.readVInt()
);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
hits.writeTo(out);
out.writeOptionalWriteable((InternalAggregations) aggregations);
out.writeOptionalWriteable(suggest);
out.writeBoolean(timedOut);
out.writeOptionalBoolean(terminatedEarly);
out.writeOptionalWriteable(profileResults);
out.writeVInt(numReducePhases);
}
}

View file

@ -18,10 +18,10 @@ import org.elasticsearch.core.Tuple;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -194,7 +194,10 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
new IllegalArgumentException() new IllegalArgumentException()
); );
} }
action.sendSearchResponse(InternalSearchResponse.EMPTY_WITH_TOTAL_HITS, phaseResults.results); action.sendSearchResponse(
new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1),
phaseResults.results
);
assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class)); assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class));
SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException) exception.get(); SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException) exception.get();
assertEquals(0, searchPhaseExecutionException.getSuppressed().length); assertEquals(0, searchPhaseExecutionException.getSuppressed().length);

View file

@ -20,7 +20,6 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.collapse.CollapseBuilder; import org.elasticsearch.search.collapse.CollapseBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
@ -89,16 +88,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
List<MultiSearchResponse.Item> mSearchResponses = new ArrayList<>(numInnerHits); List<MultiSearchResponse.Item> mSearchResponses = new ArrayList<>(numInnerHits);
for (int innerHitNum = 0; innerHitNum < numInnerHits; innerHitNum++) { for (int innerHitNum = 0; innerHitNum < numInnerHits; innerHitNum++) {
InternalSearchResponse internalSearchResponse = new InternalSearchResponse( mockSearchPhaseContext.sendSearchResponse(
collapsedHits.get(innerHitNum), new SearchResponseSections(collapsedHits.get(innerHitNum), null, null, false, null, null, 1),
null, null
null,
null,
false,
null,
1
); );
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null);
mSearchResponses.add(new MultiSearchResponse.Item(mockSearchPhaseContext.searchResponse.get(), null)); mSearchResponses.add(new MultiSearchResponse.Item(mockSearchPhaseContext.searchResponse.get(), null));
} }
@ -112,11 +105,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
SearchHit hit = new SearchHit(1, "ID"); SearchHit hit = new SearchHit(1, "ID");
hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue)));
SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") {
ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, internalSearchResponse, () -> new SearchPhase("test") {
@Override @Override
public void run() { public void run() {
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null); mockSearchPhaseContext.sendSearchResponse(new SearchResponseSections(hits, null, null, false, null, null, 1), null);
} }
}); });
@ -154,9 +146,14 @@ public class ExpandSearchPhaseTests extends ESTestCase {
@Override @Override
void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionListener<MultiSearchResponse> listener) { void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionListener<MultiSearchResponse> listener) {
assertTrue(executedMultiSearch.compareAndSet(false, true)); assertTrue(executedMultiSearch.compareAndSet(false, true));
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(collapsedHits, null, null, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, collapsedHits,
null,
null,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -182,11 +179,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
SearchHit hit2 = new SearchHit(2, "ID2"); SearchHit hit2 = new SearchHit(2, "ID2");
hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue)));
SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") {
ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, internalSearchResponse, () -> new SearchPhase("test") {
@Override @Override
public void run() { public void run() {
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null); mockSearchPhaseContext.sendSearchResponse(new SearchResponseSections(hits, null, null, false, null, null, 1), null);
} }
}); });
phase.run(); phase.run();
@ -210,11 +206,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
SearchHit hit2 = new SearchHit(2, "ID2"); SearchHit hit2 = new SearchHit(2, "ID2");
hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null)));
SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") {
ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, internalSearchResponse, () -> new SearchPhase("test") {
@Override @Override
public void run() { public void run() {
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null); mockSearchPhaseContext.sendSearchResponse(new SearchResponseSections(hits, null, null, false, null, null, 1), null);
} }
}); });
phase.run(); phase.run();
@ -238,11 +233,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
); );
SearchHits hits = new SearchHits(new SearchHit[0], new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f); SearchHits hits = new SearchHits(new SearchHit[0], new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") {
ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, internalSearchResponse, () -> new SearchPhase("test") {
@Override @Override
public void run() { public void run() {
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null); mockSearchPhaseContext.sendSearchResponse(new SearchResponseSections(hits, null, null, false, null, null, 1), null);
} }
}); });
phase.run(); phase.run();
@ -281,11 +275,10 @@ public class ExpandSearchPhaseTests extends ESTestCase {
.routing("baz"); .routing("baz");
SearchHits hits = new SearchHits(new SearchHit[0], new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f); SearchHits hits = new SearchHits(new SearchHit[0], new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") {
ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, internalSearchResponse, () -> new SearchPhase("test") {
@Override @Override
public void run() throws IOException { public void run() {
mockSearchPhaseContext.sendSearchResponse(internalSearchResponse, null); mockSearchPhaseContext.sendSearchResponse(new SearchResponseSections(hits, null, null, false, null, null, 1), null);
} }
}); });
phase.run(); phase.run();

View file

@ -17,7 +17,6 @@ import org.elasticsearch.search.SearchHitTests;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.fetch.subphase.FieldAndFormat; import org.elasticsearch.search.fetch.subphase.FieldAndFormat;
import org.elasticsearch.search.fetch.subphase.LookupField; import org.elasticsearch.search.fetch.subphase.LookupField;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import java.util.List; import java.util.List;
@ -46,8 +45,11 @@ public class FetchLookupFieldsPhaseTests extends ESTestCase {
searchHits[i] = SearchHitTests.createTestItem(randomBoolean(), randomBoolean()); searchHits[i] = SearchHitTests.createTestItem(randomBoolean(), randomBoolean());
} }
SearchHits hits = new SearchHits(searchHits, new TotalHits(numHits, TotalHits.Relation.EQUAL_TO), 1.0f); SearchHits hits = new SearchHits(searchHits, new TotalHits(numHits, TotalHits.Relation.EQUAL_TO), 1.0f);
InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, null, 1); FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(
FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(searchPhaseContext, searchResponse, null); searchPhaseContext,
new SearchResponseSections(hits, null, null, false, null, null, 1),
null
);
phase.run(); phase.run();
searchPhaseContext.assertNoFailure(); searchPhaseContext.assertNoFailure();
assertNotNull(searchPhaseContext.searchResponse.get()); assertNotNull(searchPhaseContext.searchResponse.get());
@ -95,18 +97,15 @@ public class FetchLookupFieldsPhaseTests extends ESTestCase {
} else { } else {
searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 1.0f); searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 1.0f);
} }
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(
searchHits,
null,
null,
null,
false,
null,
1
);
responses[i] = new MultiSearchResponse.Item( responses[i] = new MultiSearchResponse.Item(
new SearchResponse( new SearchResponse(
internalSearchResponse, searchHits,
null,
null,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -174,8 +173,11 @@ public class FetchLookupFieldsPhaseTests extends ESTestCase {
); );
} }
SearchHits searchHits = new SearchHits(new SearchHit[] { leftHit0, leftHit1 }, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 1.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { leftHit0, leftHit1 }, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 1.0f);
InternalSearchResponse searchResponse = new InternalSearchResponse(searchHits, null, null, null, false, null, 1); FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(
FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(searchPhaseContext, searchResponse, null); searchPhaseContext,
new SearchResponseSections(searchHits, null, null, false, null, null, 1),
null
);
phase.run(); phase.run();
assertTrue(requestSent.get()); assertTrue(requestSent.get());
searchPhaseContext.assertNoFailure(); searchPhaseContext.assertNoFailure();

View file

@ -16,7 +16,6 @@ import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables; import org.elasticsearch.core.Releasables;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
@ -83,7 +82,7 @@ public final class MockSearchPhaseContext implements SearchPhaseContext {
} }
@Override @Override
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) { public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) {
String scrollId = getRequest().scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; String scrollId = getRequest().scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null;
String searchContextId = getRequest().pointInTimeBuilder() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; String searchContextId = getRequest().pointInTimeBuilder() != null ? TransportSearchHelper.buildScrollId(queryResults) : null;
searchResponse.set( searchResponse.set(

View file

@ -19,7 +19,7 @@ import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
@ -149,8 +149,7 @@ public class MultiSearchActionTookTests extends ESTestCase {
counter.decrementAndGet(); counter.decrementAndGet();
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse( SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
0, 0,
0, 0,

View file

@ -9,7 +9,7 @@ package org.elasticsearch.action.search;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.RefCounted;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentParser;
@ -39,17 +39,14 @@ public class MultiSearchResponseTests extends ESTestCase {
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = totalShards - successfulShards; int skippedShards = totalShards - successfulShards;
SearchResponse.Clusters clusters = SearchResponseTests.randomSimpleClusters(); SearchResponse.Clusters clusters = SearchResponseTests.randomSimpleClusters();
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
SearchResponse searchResponse = new SearchResponse(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
skippedShards, skippedShards,
tookInMillis, tookInMillis,
ShardSearchFailure.EMPTY_ARRAY, ShardSearchFailure.EMPTY_ARRAY,
clusters, clusters
null
); );
items[i] = new MultiSearchResponse.Item(searchResponse, null); items[i] = new MultiSearchResponse.Item(searchResponse, null);
} }
@ -67,17 +64,14 @@ public class MultiSearchResponseTests extends ESTestCase {
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = totalShards - successfulShards; int skippedShards = totalShards - successfulShards;
SearchResponse.Clusters clusters = SearchResponseTests.randomSimpleClusters(); SearchResponse.Clusters clusters = SearchResponseTests.randomSimpleClusters();
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
SearchResponse searchResponse = new SearchResponse(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
skippedShards, skippedShards,
tookInMillis, tookInMillis,
ShardSearchFailure.EMPTY_ARRAY, ShardSearchFailure.EMPTY_ARRAY,
clusters, clusters
null
); );
items[i] = new MultiSearchResponse.Item(searchResponse, null); items[i] = new MultiSearchResponse.Item(searchResponse, null);
} else { } else {

View file

@ -21,10 +21,10 @@ import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.UUIDs;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
@ -152,7 +152,10 @@ public class SearchAsyncActionTests extends ESTestCase {
assertTrue(searchPhaseDidRun.get()); assertTrue(searchPhaseDidRun.get());
assertEquals(shardsIter.size() - numSkipped, numRequests.get()); assertEquals(shardsIter.size() - numSkipped, numRequests.get());
asyncAction.sendSearchResponse(null, null); asyncAction.sendSearchResponse(
new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1),
null
);
assertNotNull(searchResponse.get()); assertNotNull(searchResponse.get());
assertEquals(0, searchResponse.get().getFailedShards()); assertEquals(0, searchResponse.get().getFailedShards());
assertEquals(numSkipped, searchResponse.get().getSkippedShards()); assertEquals(numSkipped, searchResponse.get().getSkippedShards());
@ -695,7 +698,10 @@ public class SearchAsyncActionTests extends ESTestCase {
assertThat(latch.await(4, TimeUnit.SECONDS), equalTo(true)); assertThat(latch.await(4, TimeUnit.SECONDS), equalTo(true));
assertThat(searchPhaseDidRun.get(), equalTo(true)); assertThat(searchPhaseDidRun.get(), equalTo(true));
asyncAction.sendSearchResponse(null, null); asyncAction.sendSearchResponse(
new SearchResponseSections(SearchHits.EMPTY_WITH_TOTAL_HITS, null, null, false, null, null, 1),
null
);
assertNotNull(searchResponse.get()); assertNotNull(searchResponse.get());
assertThat(searchResponse.get().getSkippedShards(), equalTo(numUnavailableSkippedShards)); assertThat(searchResponse.get().getSkippedShards(), equalTo(numUnavailableSkippedShards));
assertThat(searchResponse.get().getFailedShards(), equalTo(0)); assertThat(searchResponse.get().getFailedShards(), equalTo(0));
@ -772,7 +778,23 @@ public class SearchAsyncActionTests extends ESTestCase {
final Set<ShardId> queried = new HashSet<>(); final Set<ShardId> queried = new HashSet<>();
TestSearchResponse() { TestSearchResponse() {
super(InternalSearchResponse.EMPTY_WITH_TOTAL_HITS, null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, Clusters.EMPTY, null); super(
SearchHits.EMPTY_WITH_TOTAL_HITS,
null,
null,
false,
null,
null,
1,
null,
0,
0,
0,
0L,
ShardSearchFailure.EMPTY_ARRAY,
Clusters.EMPTY,
null
);
} }
} }

View file

@ -50,7 +50,6 @@ import org.elasticsearch.search.aggregations.metrics.Max;
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.profile.ProfileResult; import org.elasticsearch.search.profile.ProfileResult;
@ -294,7 +293,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
profile profile
); );
try { try {
InternalSearchResponse mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults); SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults);
if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) { if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) {
assertNull(mergedResponse.hits.getTotalHits()); assertNull(mergedResponse.hits.getTotalHits());
} else { } else {
@ -412,7 +411,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
false false
); );
try { try {
InternalSearchResponse mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults); SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults);
if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) { if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) {
assertNull(mergedResponse.hits.getTotalHits()); assertNull(mergedResponse.hits.getTotalHits());
} else { } else {

View file

@ -21,6 +21,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
@ -29,7 +30,6 @@ import org.elasticsearch.search.aggregations.bucket.range.InternalDateRange;
import org.elasticsearch.search.aggregations.bucket.range.Range; import org.elasticsearch.search.aggregations.bucket.range.Range;
import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.aggregations.metrics.Max;
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileResultsTests; import org.elasticsearch.search.profile.SearchProfileResultsTests;
@ -108,8 +108,7 @@ public class SearchResponseMergerTests extends ESTestCase {
) )
) { ) {
for (int i = 0; i < numResponses; i++) { for (int i = 0; i < numResponses; i++) {
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
1, 1,
1, 1,
@ -169,8 +168,7 @@ public class SearchResponseMergerTests extends ESTestCase {
shardSearchFailures[j] = failure; shardSearchFailures[j] = failure;
priorityQueue.add(Tuple.tuple(searchShardTarget, failure)); priorityQueue.add(Tuple.tuple(searchShardTarget, failure));
} }
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
1, 1,
1, 1,
@ -231,8 +229,7 @@ public class SearchResponseMergerTests extends ESTestCase {
shardSearchFailures[j] = failure; shardSearchFailures[j] = failure;
priorityQueue.add(Tuple.tuple(shardId, failure)); priorityQueue.add(Tuple.tuple(shardId, failure));
} }
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
1, 1,
1, 1,
@ -291,8 +288,7 @@ public class SearchResponseMergerTests extends ESTestCase {
shardSearchFailures[j] = shardSearchFailure; shardSearchFailures[j] = shardSearchFailure;
expectedFailures.add(shardSearchFailure); expectedFailures.add(shardSearchFailure);
} }
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
1, 1,
1, 1,
@ -335,9 +331,14 @@ public class SearchResponseMergerTests extends ESTestCase {
SearchProfileResults profile = SearchProfileResultsTests.createTestItem(); SearchProfileResults profile = SearchProfileResultsTests.createTestItem();
expectedProfile.putAll(profile.getShardResults()); expectedProfile.putAll(profile.getShardResults());
SearchHits searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN); SearchHits searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits, null, null, profile, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, searchHits,
null,
null,
false,
null,
profile,
1,
null, null,
1, 1,
1, 1,
@ -408,9 +409,14 @@ public class SearchResponseMergerTests extends ESTestCase {
suggestions.add(completionSuggestion); suggestions.add(completionSuggestion);
Suggest suggest = new Suggest(suggestions); Suggest suggest = new Suggest(suggestions);
SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN); SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits, null, suggest, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, searchHits,
null,
suggest,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -489,9 +495,14 @@ public class SearchResponseMergerTests extends ESTestCase {
suggestions.add(completionSuggestion); suggestions.add(completionSuggestion);
Suggest suggest = new Suggest(suggestions); Suggest suggest = new Suggest(suggestions);
SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN); SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits, null, suggest, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, searchHits,
null,
suggest,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -566,9 +577,14 @@ public class SearchResponseMergerTests extends ESTestCase {
) { ) {
for (Max max : Arrays.asList(max1, max2)) { for (Max max : Arrays.asList(max1, max2)) {
InternalAggregations aggs = InternalAggregations.from(Arrays.asList(max)); InternalAggregations aggs = InternalAggregations.from(Arrays.asList(max));
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits, aggs, null, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, searchHits,
aggs,
null,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -630,9 +646,14 @@ public class SearchResponseMergerTests extends ESTestCase {
InternalDateRange range = factory.create(rangeAggName, singletonList(bucket), DocValueFormat.RAW, false, emptyMap()); InternalDateRange range = factory.create(rangeAggName, singletonList(bucket), DocValueFormat.RAW, false, emptyMap());
InternalAggregations aggs = InternalAggregations.from(Arrays.asList(range, max)); InternalAggregations aggs = InternalAggregations.from(Arrays.asList(range, max));
SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN); SearchHits searchHits = new SearchHits(new SearchHit[0], null, Float.NaN);
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchHits, aggs, null, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, searchHits,
aggs,
null,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,
@ -787,18 +808,14 @@ public class SearchResponseMergerTests extends ESTestCase {
Boolean terminatedEarly = frequently() ? null : true; Boolean terminatedEarly = frequently() ? null : true;
expectedTerminatedEarly = expectedTerminatedEarly == null ? terminatedEarly : expectedTerminatedEarly; expectedTerminatedEarly = expectedTerminatedEarly == null ? terminatedEarly : expectedTerminatedEarly;
InternalSearchResponse internalSearchResponse = new InternalSearchResponse( SearchResponse searchResponse = new SearchResponse(
searchHits, searchHits,
null, null,
null, null,
null,
timedOut, timedOut,
terminatedEarly, terminatedEarly,
numReducePhases null,
); numReducePhases,
SearchResponse searchResponse = new SearchResponse(
internalSearchResponse,
null, null,
total, total,
successful, successful,
@ -937,9 +954,14 @@ public class SearchResponseMergerTests extends ESTestCase {
null, null,
null null
); );
InternalSearchResponse response = new InternalSearchResponse(searchHits, null, null, null, false, false, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
response, searchHits,
null,
null,
false,
false,
null,
1,
null, null,
1, 1,
1, 1,
@ -963,9 +985,14 @@ public class SearchResponseMergerTests extends ESTestCase {
null, null,
null null
); );
InternalSearchResponse response = new InternalSearchResponse(empty, null, null, null, false, false, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
response, empty,
null,
null,
false,
false,
null,
1,
null, null,
1, 1,
1, 1,
@ -1015,9 +1042,14 @@ public class SearchResponseMergerTests extends ESTestCase {
expectedTotalHits = new TotalHits(Math.min(previousValue + totalHits.value, trackTotalHitsUpTo), totalHitsRelation); expectedTotalHits = new TotalHits(Math.min(previousValue + totalHits.value, trackTotalHitsUpTo), totalHitsRelation);
} }
SearchHits empty = new SearchHits(new SearchHit[0], totalHits, Float.NaN, null, null, null); SearchHits empty = new SearchHits(new SearchHit[0], totalHits, Float.NaN, null, null, null);
InternalSearchResponse response = new InternalSearchResponse(empty, null, null, null, false, false, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
response, empty,
null,
null,
false,
false,
null,
1,
null, null,
1, 1,
1, 1,

View file

@ -25,9 +25,9 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchHitsTests; import org.elasticsearch.search.SearchHitsTests;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.aggregations.AggregationsTests; import org.elasticsearch.search.aggregations.AggregationsTests;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.profile.SearchProfileResultsTests; import org.elasticsearch.search.profile.SearchProfileResultsTests;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
@ -107,42 +107,44 @@ public class SearchResponseTests extends ESTestCase {
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, totalShards);
InternalSearchResponse internalSearchResponse;
if (minimal == false) {
SearchHits hits = SearchHitsTests.createTestItem(true, true);
InternalAggregations aggregations = aggregationsTests.createTestInstance();
Suggest suggest = SuggestTests.createTestItem();
SearchProfileResults profileResults = SearchProfileResultsTests.createTestItem();
internalSearchResponse = new InternalSearchResponse(
hits,
aggregations,
suggest,
profileResults,
timedOut,
terminatedEarly,
numReducePhases
);
} else {
internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
}
SearchResponse.Clusters clusters; SearchResponse.Clusters clusters;
if (minimal) { if (minimal) {
clusters = randomSimpleClusters(); clusters = randomSimpleClusters();
} else { } else {
clusters = randomClusters(); clusters = randomClusters();
} }
if (minimal == false) {
return new SearchResponse( SearchHits hits = SearchHitsTests.createTestItem(true, true);
internalSearchResponse, InternalAggregations aggregations = aggregationsTests.createTestInstance();
null, Suggest suggest = SuggestTests.createTestItem();
totalShards, SearchProfileResults profileResults = SearchProfileResultsTests.createTestItem();
successfulShards, return new SearchResponse(
skippedShards, hits,
tookInMillis, aggregations,
shardSearchFailures, suggest,
clusters timedOut,
); terminatedEarly,
profileResults,
numReducePhases,
null,
totalShards,
successfulShards,
skippedShards,
tookInMillis,
shardSearchFailures,
clusters
);
} else {
return SearchResponseUtils.emptyWithTotalHits(
null,
totalShards,
successfulShards,
skippedShards,
tookInMillis,
shardSearchFailures,
clusters
);
}
} }
/** /**
@ -381,15 +383,13 @@ public class SearchResponseTests extends ESTestCase {
SearchHit[] hits = new SearchHit[] { hit }; SearchHit[] hits = new SearchHit[] { hit };
{ {
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
new InternalSearchResponse( new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f),
new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f), null,
null, null,
null, false,
null, null,
false, null,
null, 1,
1
),
null, null,
0, 0,
0, 0,
@ -425,15 +425,13 @@ public class SearchResponseTests extends ESTestCase {
} }
{ {
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
new InternalSearchResponse( new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f),
new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f), null,
null, null,
null, false,
null, null,
false, null,
null, 1,
1
),
null, null,
0, 0,
0, 0,
@ -477,15 +475,13 @@ public class SearchResponseTests extends ESTestCase {
} }
{ {
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
new InternalSearchResponse( new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f),
new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f), null,
null, null,
null, false,
null, null,
false, null,
null, 1,
1
),
null, null,
20, 20,
9, 9,
@ -654,8 +650,7 @@ public class SearchResponseTests extends ESTestCase {
} }
public void testToXContentEmptyClusters() throws IOException { public void testToXContentEmptyClusters() throws IOException {
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
1, 1,
1, 1,

View file

@ -23,7 +23,7 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -166,8 +166,7 @@ public class TransportMultiSearchActionTests extends ESTestCase {
final ExecutorService executorService = rarely() ? rarelyExecutor : commonExecutor; final ExecutorService executorService = rarely() ? rarelyExecutor : commonExecutor;
executorService.execute(() -> { executorService.execute(() -> {
counter.decrementAndGet(); counter.decrementAndGet();
var response = new SearchResponse( var response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null, null,
0, 0,
0, 0,

View file

@ -69,7 +69,6 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.collapse.CollapseBuilder; import org.elasticsearch.search.collapse.CollapseBuilder;
import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortBuilders;
@ -481,16 +480,23 @@ public class TransportSearchActionTests extends ESTestCase {
} }
private static SearchResponse emptySearchResponse() { private static SearchResponse emptySearchResponse() {
InternalSearchResponse response = new InternalSearchResponse( return new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN),
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
1 null,
1,
null,
1,
1,
0,
100,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY,
null
); );
return new SearchResponse(response, null, 1, 1, 0, 100, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY, null);
} }
public void testCCSRemoteReduceMergeFails() throws Exception { public void testCCSRemoteReduceMergeFails() throws Exception {

View file

@ -45,7 +45,6 @@ import org.elasticsearch.mocksocket.MockServerSocket;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
@ -156,17 +155,14 @@ public class RemoteClusterConnectionTests extends ESTestCase {
} else { } else {
searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN); searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN);
} }
InternalSearchResponse response = new InternalSearchResponse( SearchResponse searchResponse = new SearchResponse(
searchHits, searchHits,
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
1 null,
); 1,
SearchResponse searchResponse = new SearchResponse(
response,
null, null,
1, 1,
1, 1,

View file

@ -10,6 +10,7 @@ package org.elasticsearch.search;
import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.TotalHits;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.client.Response; import org.elasticsearch.client.Response;
import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.ESRestTestCase;
@ -36,4 +37,31 @@ public enum SearchResponseUtils {
return SearchResponse.fromXContent(parser); return SearchResponse.fromXContent(parser);
} }
} }
public static SearchResponse emptyWithTotalHits(
String scrollId,
int totalShards,
int successfulShards,
int skippedShards,
long tookInMillis,
ShardSearchFailure[] shardFailures,
SearchResponse.Clusters clusters
) {
return new SearchResponse(
SearchHits.EMPTY_WITH_TOTAL_HITS,
null,
null,
false,
null,
null,
1,
scrollId,
totalShards,
successfulShards,
skippedShards,
tookInMillis,
shardFailures,
clusters
);
}
} }

View file

@ -18,7 +18,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse; import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
import org.elasticsearch.xpack.core.search.action.AsyncStatusResponse; import org.elasticsearch.xpack.core.search.action.AsyncStatusResponse;
@ -156,18 +155,15 @@ class MutableSearchResponse {
} }
private SearchResponse buildResponse(long taskStartTimeNanos, InternalAggregations reducedAggs) { private SearchResponse buildResponse(long taskStartTimeNanos, InternalAggregations reducedAggs) {
InternalSearchResponse internal = new InternalSearchResponse( long tookInMillis = TimeValue.timeValueNanos(System.nanoTime() - taskStartTimeNanos).getMillis();
return new SearchResponse(
new SearchHits(SearchHits.EMPTY, totalHits, Float.NaN), new SearchHits(SearchHits.EMPTY, totalHits, Float.NaN),
reducedAggs, reducedAggs,
null, null,
false,
false,
null, null,
false, reducePhase,
false,
reducePhase
);
long tookInMillis = TimeValue.timeValueNanos(System.nanoTime() - taskStartTimeNanos).getMillis();
return new SearchResponse(
internal,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,

View file

@ -11,7 +11,6 @@ import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersion;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -25,8 +24,8 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.script.ScriptException; import org.elasticsearch.script.ScriptException;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterAware;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
@ -129,15 +128,13 @@ public class AsyncSearchResponseTests extends ESTestCase {
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, successfulShards); int skippedShards = randomIntBetween(0, successfulShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
SearchResponse.Clusters clusters; SearchResponse.Clusters clusters;
if (ccs) { if (ccs) {
clusters = createCCSClusterObjects(20, 19, true, 10, 1, 2); clusters = createCCSClusterObjects(20, 19, true, 10, 1, 2);
} else { } else {
clusters = SearchResponse.Clusters.EMPTY; clusters = SearchResponse.Clusters.EMPTY;
} }
return new SearchResponse( return SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -211,9 +208,14 @@ public class AsyncSearchResponseTests extends ESTestCase {
long expectedCompletionTime = startTimeMillis + took; long expectedCompletionTime = startTimeMillis + took;
SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS;
SearchResponseSections sections = new SearchResponseSections(hits, null, null, false, null, null, 2);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
sections, hits,
null,
null,
false,
null,
null,
2,
null, null,
10, 10,
9, 9,
@ -316,11 +318,25 @@ public class AsyncSearchResponseTests extends ESTestCase {
long took = 22968L; long took = 22968L;
SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS;
SearchResponseSections sections = new SearchResponseSections(hits, null, null, false, null, null, 2);
SearchResponse.Clusters clusters = createCCSClusterObjects(3, 3, true); SearchResponse.Clusters clusters = createCCSClusterObjects(3, 3, true);
SearchResponse searchResponse = new SearchResponse(sections, null, 10, 9, 1, took, ShardSearchFailure.EMPTY_ARRAY, clusters); SearchResponse searchResponse = new SearchResponse(
hits,
null,
null,
false,
null,
null,
2,
null,
10,
9,
1,
took,
ShardSearchFailure.EMPTY_ARRAY,
clusters
);
AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse( AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse(
"id", "id",
@ -462,7 +478,6 @@ public class AsyncSearchResponseTests extends ESTestCase {
long expectedCompletionTime = startTimeMillis + took; long expectedCompletionTime = startTimeMillis + took;
SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS;
SearchResponseSections sections = new SearchResponseSections(hits, null, null, true, null, null, 2);
SearchResponse.Clusters clusters = createCCSClusterObjects(4, 3, true); SearchResponse.Clusters clusters = createCCSClusterObjects(4, 3, true);
SearchResponse.Cluster updated = clusters.swapCluster( SearchResponse.Cluster updated = clusters.swapCluster(
@ -532,7 +547,22 @@ public class AsyncSearchResponseTests extends ESTestCase {
); );
assertNotNull("Set cluster failed for cluster " + cluster2.getClusterAlias(), updated); assertNotNull("Set cluster failed for cluster " + cluster2.getClusterAlias(), updated);
SearchResponse searchResponse = new SearchResponse(sections, null, 10, 9, 1, took, new ShardSearchFailure[0], clusters); SearchResponse searchResponse = new SearchResponse(
hits,
null,
null,
true,
null,
null,
2,
null,
10,
9,
1,
took,
new ShardSearchFailure[0],
clusters
);
AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse( AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse(
"id", "id",
@ -659,9 +689,14 @@ public class AsyncSearchResponseTests extends ESTestCase {
long took = 22968L; long took = 22968L;
SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS;
SearchResponseSections sections = new SearchResponseSections(hits, null, null, false, null, null, 2);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
sections, hits,
null,
null,
false,
null,
null,
2,
null, null,
10, 10,
9, 9,

View file

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.client.NoOpClient; import org.elasticsearch.test.client.NoOpClient;
@ -405,17 +404,14 @@ public class AsyncSearchTaskTests extends ESTestCase {
int skippedShards, int skippedShards,
ShardSearchFailure... failures ShardSearchFailure... failures
) { ) {
InternalSearchResponse response = new InternalSearchResponse( return new SearchResponse(
SearchHits.EMPTY_WITH_TOTAL_HITS, SearchHits.EMPTY_WITH_TOTAL_HITS,
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
1 null,
); 1,
return new SearchResponse(
response,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,

View file

@ -13,7 +13,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.AbstractWireSerializingTestCase; import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentBuilder;
@ -316,10 +316,8 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, successfulShards); int skippedShards = randomIntBetween(0, successfulShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 99, 1); SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 99, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -343,10 +341,8 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, successfulShards); int skippedShards = randomIntBetween(0, successfulShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -370,7 +366,6 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, successfulShards); int skippedShards = randomIntBetween(0, successfulShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
int totalClusters; int totalClusters;
int successfulClusters; int successfulClusters;
@ -390,8 +385,7 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
skippedClusters = totalClusters - (successfulClusters + partial); skippedClusters = totalClusters - (successfulClusters + partial);
clusters = AsyncSearchResponseTests.createCCSClusterObjects(80, 80, true, successfulClusters, skippedClusters, partial); clusters = AsyncSearchResponseTests.createCCSClusterObjects(80, 80, true, successfulClusters, skippedClusters, partial);
} }
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,
@ -421,7 +415,6 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
int successfulShards = randomIntBetween(0, totalShards); int successfulShards = randomIntBetween(0, totalShards);
int skippedShards = randomIntBetween(0, successfulShards); int skippedShards = randomIntBetween(0, successfulShards);
InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS;
int successful = randomInt(10); int successful = randomInt(10);
int partial = randomInt(10); int partial = randomInt(10);
int skipped = randomInt(10); int skipped = randomInt(10);
@ -437,8 +430,7 @@ public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<As
} }
SearchResponse.Clusters clusters = AsyncSearchResponseTests.createCCSClusterObjects(100, 99, true, successful, skipped, partial); SearchResponse.Clusters clusters = AsyncSearchResponseTests.createCCSClusterObjects(100, 99, true, successful, skipped, partial);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
internalSearchResponse,
null, null,
totalShards, totalShards,
successfulShards, successfulShards,

View file

@ -24,7 +24,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TransportVersionUtils; import org.elasticsearch.test.TransportVersionUtils;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -248,16 +248,7 @@ public class ClientHelperTests extends ESTestCase {
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
searchFuture.onResponse( searchFuture.onResponse(
new SearchResponse( SearchResponseUtils.emptyWithTotalHits(null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY)
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null,
0,
0,
0,
0L,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
)
); );
when(client.search(any())).thenReturn(searchFuture); when(client.search(any())).thenReturn(searchFuture);
assertExecutionWithOrigin(Collections.emptyMap(), client); assertExecutionWithOrigin(Collections.emptyMap(), client);
@ -272,16 +263,7 @@ public class ClientHelperTests extends ESTestCase {
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
searchFuture.onResponse( searchFuture.onResponse(
new SearchResponse( SearchResponseUtils.emptyWithTotalHits(null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY)
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null,
0,
0,
0,
0L,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
)
); );
when(client.search(any())).thenReturn(searchFuture); when(client.search(any())).thenReturn(searchFuture);
Map<String, String> headers = Map.of( Map<String, String> headers = Map.of(
@ -307,16 +289,7 @@ public class ClientHelperTests extends ESTestCase {
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
searchFuture.onResponse( searchFuture.onResponse(
new SearchResponse( SearchResponseUtils.emptyWithTotalHits(null, 0, 0, 0, 0L, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY)
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
null,
0,
0,
0,
0L,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
)
); );
when(client.search(any())).thenReturn(searchFuture); when(client.search(any())).thenReturn(searchFuture);
Map<String, String> unrelatedHeaders = Map.of(randomAlphaOfLength(10), "anything"); Map<String, String> unrelatedHeaders = Map.of(randomAlphaOfLength(10), "anything");

View file

@ -14,7 +14,6 @@ import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -116,18 +115,24 @@ public class AsyncTwoPhaseIndexerTests extends ESTestCase {
return; return;
} }
final SearchResponseSections sections = new SearchResponseSections(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
null,
null,
false,
null,
null,
1
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
nextPhase, nextPhase,
new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null) new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
null,
null,
false,
null,
null,
1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
)
); );
} }
@ -256,15 +261,6 @@ public class AsyncTwoPhaseIndexerTests extends ESTestCase {
@Override @Override
protected void doNextSearch(long waitTimeInNanos, ActionListener<SearchResponse> nextPhase) { protected void doNextSearch(long waitTimeInNanos, ActionListener<SearchResponse> nextPhase) {
++searchOps; ++searchOps;
final SearchResponseSections sections = new SearchResponseSections(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
null,
null,
false,
null,
null,
1
);
if (processOps == 3) { if (processOps == 3) {
awaitForLatch(); awaitForLatch();
@ -272,7 +268,22 @@ public class AsyncTwoPhaseIndexerTests extends ESTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
nextPhase, nextPhase,
new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null) new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
null,
null,
false,
null,
null,
1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
)
); );
} }

View file

@ -55,7 +55,6 @@ import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.DeprecationHandler;
@ -305,7 +304,13 @@ public class EnrichShardMultiSearchAction extends ActionType<MultiSearchResponse
private static SearchResponse createSearchResponse(TopDocs topDocs, SearchHit[] hits) { private static SearchResponse createSearchResponse(TopDocs topDocs, SearchHit[] hits) {
SearchHits searchHits = new SearchHits(hits, topDocs.totalHits, 0); SearchHits searchHits = new SearchHits(hits, topDocs.totalHits, 0);
return new SearchResponse( return new SearchResponse(
new InternalSearchResponse(searchHits, null, null, null, false, null, 0), searchHits,
null,
null,
false,
null,
null,
0,
null, null,
1, 1,
1, 1,

View file

@ -28,7 +28,6 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -261,15 +260,13 @@ public class EnrichProcessorFactoryTests extends ESTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f), InternalAggregations.EMPTY,
InternalAggregations.EMPTY, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,

View file

@ -25,7 +25,6 @@ import org.elasticsearch.index.query.MatchQueryBuilder;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -373,16 +372,22 @@ public class CoordinatorTests extends ESTestCase {
} }
private static SearchResponse emptySearchResponse() { private static SearchResponse emptySearchResponse() {
InternalSearchResponse response = new InternalSearchResponse( return new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN),
InternalAggregations.EMPTY, InternalAggregations.EMPTY,
null, null,
null,
false, false,
null, null,
1 null,
1,
null,
1,
1,
0,
100,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
); );
return new SearchResponse(response, null, 1, 1, 0, 100, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY);
} }
private class MockLookupFunction implements BiConsumer<MultiSearchRequest, BiConsumer<MultiSearchResponse, Exception>> { private class MockLookupFunction implements BiConsumer<MultiSearchRequest, BiConsumer<MultiSearchResponse, Exception>> {

View file

@ -13,7 +13,6 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponse.Clusters; import org.elasticsearch.action.search.SearchResponse.Clusters;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.DocValueFormat;
@ -83,8 +82,10 @@ public class ImplicitTiebreakerTests extends ESTestCase {
) )
); );
SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f);
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0); ActionListener.respondAndRelease(
ActionListener.respondAndRelease(l, new SearchResponse(internal, null, 0, 1, 0, 0, null, Clusters.EMPTY)); l,
new SearchResponse(searchHits, null, null, false, false, null, 0, null, 0, 1, 0, 0, null, Clusters.EMPTY)
);
} }
@Override @Override

View file

@ -14,7 +14,6 @@ import org.apache.lucene.search.TotalHits.Relation;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponse.Clusters; import org.elasticsearch.action.search.SearchResponse.Clusters;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.document.DocumentField;
@ -221,8 +220,10 @@ public class SequenceSpecTests extends ESTestCase {
new TotalHits(eah.hits.size(), Relation.EQUAL_TO), new TotalHits(eah.hits.size(), Relation.EQUAL_TO),
0.0f 0.0f
); );
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0); ActionListener.respondAndRelease(
ActionListener.respondAndRelease(l, new SearchResponse(internal, null, 0, 1, 0, 0, null, Clusters.EMPTY)); l,
new SearchResponse(searchHits, null, null, false, false, null, 0, null, 0, 1, 0, 0, null, Clusters.EMPTY)
);
} }
@Override @Override

View file

@ -18,7 +18,6 @@ import org.elasticsearch.action.search.OpenPointInTimeResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponse.Clusters; import org.elasticsearch.action.search.SearchResponse.Clusters;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.CircuitBreakingException;
@ -222,12 +221,16 @@ public class CircuitBreakerTests extends ESTestCase {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
<Response extends ActionResponse> void handleSearchRequest(ActionListener<Response> listener, SearchRequest searchRequest) { <Response extends ActionResponse> void handleSearchRequest(ActionListener<Response> listener, SearchRequest searchRequest) {
Aggregations aggs = new Aggregations(List.of(newInternalComposite())); Aggregations aggs = new Aggregations(List.of(newInternalComposite()));
SearchResponseSections internal = new SearchResponseSections(null, aggs, null, false, false, null, 0);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
internal, null,
aggs,
null,
false,
false,
null,
0,
null, null,
2, 2,
0, 0,

View file

@ -18,7 +18,6 @@ import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.OpenPointInTimeResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker;
@ -243,9 +242,14 @@ public class PITAwareQueryClientTests extends ESTestCase {
); );
SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 0.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 0.0f);
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0);
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
internal, searchHits,
null,
null,
false,
false,
null,
0,
null, null,
2, 2,
0, 0,

View file

@ -21,7 +21,6 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponse.Clusters; import org.elasticsearch.action.search.SearchResponse.Clusters;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.ParsingException;
@ -46,7 +45,6 @@ import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.SearchSortValues; import org.elasticsearch.search.SearchSortValues;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
@ -114,8 +112,10 @@ public class CircuitBreakerTests extends ESTestCase {
new SearchSortValues(new Long[] { (long) ordinal, 1L }, new DocValueFormat[] { DocValueFormat.RAW, DocValueFormat.RAW }) new SearchSortValues(new Long[] { (long) ordinal, 1L }, new DocValueFormat[] { DocValueFormat.RAW, DocValueFormat.RAW })
); );
SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f);
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0); ActionListener.respondAndRelease(
ActionListener.respondAndRelease(l, new SearchResponse(internal, null, 0, 1, 0, 0, null, Clusters.EMPTY)); l,
new SearchResponse(searchHits, null, null, false, false, null, 0, null, 0, 1, 0, 0, null, Clusters.EMPTY)
);
} }
@Override @Override
@ -431,9 +431,14 @@ public class CircuitBreakerTests extends ESTestCase {
); );
SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f);
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0);
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
internal, searchHits,
null,
null,
false,
false,
null,
0,
null, null,
2, 2,
0, 0,
@ -477,11 +482,16 @@ public class CircuitBreakerTests extends ESTestCase {
new SearchSortValues(new Long[] { (long) ordinal, 1L }, new DocValueFormat[] { DocValueFormat.RAW, DocValueFormat.RAW }) new SearchSortValues(new Long[] { (long) ordinal, 1L }, new DocValueFormat[] { DocValueFormat.RAW, DocValueFormat.RAW })
); );
SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f); SearchHits searchHits = new SearchHits(new SearchHit[] { searchHit }, new TotalHits(1, Relation.EQUAL_TO), 0.0f);
SearchResponseSections internal = new SearchResponseSections(searchHits, null, null, false, false, null, 0);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
internal, searchHits,
null,
null,
false,
false,
null,
0,
null, null,
2, 2,
0, 0,
@ -509,15 +519,13 @@ public class CircuitBreakerTests extends ESTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
null, null,
2, 2,
1, 1,

View file

@ -30,7 +30,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.MockUtils; import org.elasticsearch.test.MockUtils;
@ -116,9 +115,14 @@ public class TransportGetPipelineActionTests extends ESTestCase {
* Test that the explicit and wildcard IDs are requested. * Test that the explicit and wildcard IDs are requested.
*/ */
public void testGetPipelinesByExplicitAndWildcardIds() { public void testGetPipelinesByExplicitAndWildcardIds() {
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(prepareSearchHits(), null, null, null, false, null, 1);
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
internalSearchResponse, prepareSearchHits(),
null,
null,
false,
null,
null,
1,
null, null,
1, 1,
1, 1,

View file

@ -33,6 +33,8 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException; import org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.ClientHelper;
@ -75,23 +77,28 @@ public class ResultsPersisterServiceTests extends ESTestCase {
// Constants for searchWithRetry tests // Constants for searchWithRetry tests
private static final SearchRequest SEARCH_REQUEST = new SearchRequest("my-index"); private static final SearchRequest SEARCH_REQUEST = new SearchRequest("my-index");
private static final SearchResponse SEARCH_RESPONSE_SUCCESS = new SearchResponse( public static final SearchResponse SEARCH_RESPONSE_SUCCESS = SearchResponseUtils.emptyWithTotalHits(
null,
null, null,
1, 1,
1, 1,
0, 0,
0, 1L,
ShardSearchFailure.EMPTY_ARRAY, ShardSearchFailure.EMPTY_ARRAY,
null null
); );
private static final SearchResponse SEARCH_RESPONSE_FAILURE = new SearchResponse( public static final SearchResponse SEARCH_RESPONSE_FAILURE = new SearchResponse(
SearchHits.EMPTY_WITHOUT_TOTAL_HITS,
null, null,
null, null,
false,
null,
null,
1,
null,
1, 1,
0, 0,
0, 0,
0, 1L,
ShardSearchFailure.EMPTY_ARRAY, ShardSearchFailure.EMPTY_ARRAY,
null null
); );
@ -418,4 +425,5 @@ public class ResultsPersisterServiceTests extends ESTestCase {
}).when(tp).schedule(any(Runnable.class), any(TimeValue.class), any(Executor.class)); }).when(tp).schedule(any(Runnable.class), any(TimeValue.class), any(Executor.class));
return new ResultsPersisterService(tp, client, clusterService, Settings.EMPTY); return new ResultsPersisterService(tp, client, clusterService, Settings.EMPTY);
} }
} }

View file

@ -34,7 +34,6 @@ import org.elasticsearch.search.aggregations.metrics.Max;
import org.elasticsearch.search.aggregations.metrics.Min; import org.elasticsearch.search.aggregations.metrics.Min;
import org.elasticsearch.search.aggregations.metrics.Sum; import org.elasticsearch.search.aggregations.metrics.Sum;
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.RollupField;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
@ -340,20 +339,15 @@ public class RollupResponseTranslator {
isTerminatedEarly = isTerminatedEarly && liveResponse.isTerminatedEarly(); isTerminatedEarly = isTerminatedEarly && liveResponse.isTerminatedEarly();
numReducePhases += liveResponse.getNumReducePhases(); numReducePhases += liveResponse.getNumReducePhases();
} }
// Shard failures are ignored atm, so returning an empty array is fine
InternalSearchResponse combinedInternal = new InternalSearchResponse( return new SearchResponse(
SearchHits.EMPTY_WITH_TOTAL_HITS, SearchHits.EMPTY_WITH_TOTAL_HITS,
aggs, aggs,
null, null,
null,
isTimedOut, isTimedOut,
isTerminatedEarly, isTerminatedEarly,
numReducePhases null,
); numReducePhases,
// Shard failures are ignored atm, so returning an empty array is fine
return new SearchResponse(
combinedInternal,
null, null,
totalShards, totalShards,
sucessfulShards, sucessfulShards,

View file

@ -72,7 +72,6 @@ import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.PipelineTree; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.PipelineTree;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.InternalAggregationTestCase; import org.elasticsearch.test.InternalAggregationTestCase;
import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.RollupField;
@ -516,15 +515,13 @@ public class RollupResponseTranslationTests extends AggregatorTestCase {
// TODO SearchResponse.Clusters is not public, using null for now. Should fix upstream. // TODO SearchResponse.Clusters is not public, using null for now. Should fix upstream.
MultiSearchResponse.Item unrolledItem = new MultiSearchResponse.Item( MultiSearchResponse.Item unrolledItem = new MultiSearchResponse.Item(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( null,
null, InternalAggregations.from(Collections.singletonList(responses.get(0))),
InternalAggregations.from(Collections.singletonList(responses.get(0))), null,
null, false,
null, false,
false, null,
false, 1,
1
),
null, null,
1, 1,
1, 1,
@ -537,15 +534,13 @@ public class RollupResponseTranslationTests extends AggregatorTestCase {
); );
MultiSearchResponse.Item rolledItem = new MultiSearchResponse.Item( MultiSearchResponse.Item rolledItem = new MultiSearchResponse.Item(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( null,
null, InternalAggregations.from(Collections.singletonList(responses.get(1))),
InternalAggregations.from(Collections.singletonList(responses.get(1))), null,
null, false,
null, false,
false, null,
false, 1,
1
),
null, null,
1, 1,
1, 1,

View file

@ -27,7 +27,6 @@ import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.Rounding; import org.elasticsearch.common.Rounding;
import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateFormatter;
@ -866,16 +865,22 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
} catch (IOException e) { } catch (IOException e) {
listener.onFailure(e); listener.onFailure(e);
} }
SearchResponseSections sections = new SearchResponseSections( SearchResponse response = new SearchResponse(
null, null,
new Aggregations(Collections.singletonList(result)), new Aggregations(Collections.singletonList(result)),
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
); );
SearchResponse response = new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null);
listener.onResponse(response); listener.onResponse(response);
} }

View file

@ -14,7 +14,6 @@ import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
@ -106,16 +105,22 @@ public class RollupIndexerStateTests extends ESTestCase {
return null; return null;
} }
})); }));
final SearchResponseSections sections = new SearchResponseSections( final SearchResponse response = new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs, aggs,
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
new ShardSearchFailure[0],
null
); );
final SearchResponse response = new SearchResponse(sections, null, 1, 1, 0, 0, new ShardSearchFailure[0], null);
nextPhase.onResponse(response); nextPhase.onResponse(response);
} }
@ -472,18 +477,24 @@ public class RollupIndexerStateTests extends ESTestCase {
return null; return null;
} }
})); }));
final SearchResponseSections sections = new SearchResponseSections(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs,
null,
false,
null,
null,
1
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
nextPhase, nextPhase,
new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null) new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs,
null,
false,
null,
null,
1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
)
); );
} }
@ -685,16 +696,22 @@ public class RollupIndexerStateTests extends ESTestCase {
return null; return null;
} }
})); }));
final SearchResponseSections sections = new SearchResponseSections( return new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs, aggs,
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
); );
return new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null);
}; };
Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100); Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100);
@ -809,16 +826,22 @@ public class RollupIndexerStateTests extends ESTestCase {
return null; return null;
} }
})); }));
final SearchResponseSections sections = new SearchResponseSections( return new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs, aggs,
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
); );
return new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null);
}; };
Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100); Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100);
@ -982,16 +1005,22 @@ public class RollupIndexerStateTests extends ESTestCase {
return null; return null;
} }
})); }));
final SearchResponseSections sections = new SearchResponseSections( return new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0), new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), 0),
aggs, aggs,
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
); );
return new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null);
}; };
Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> { Function<BulkRequest, BulkResponse> bulkFunction = bulkRequest -> {

View file

@ -38,7 +38,6 @@ import org.elasticsearch.core.Tuple;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.rest.ObjectPath; import org.elasticsearch.test.rest.ObjectPath;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
@ -1152,15 +1151,13 @@ public class CrossClusterAccessHeadersForCcsRestIT extends SecurityOnTrialLicens
); );
channel.sendResponse( channel.sendResponse(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN),
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), InternalAggregations.EMPTY,
InternalAggregations.EMPTY, null,
null, false,
null, null,
false, null,
null, 1,
1
),
null, null,
1, 1,
1, 1,

View file

@ -37,7 +37,6 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.SecuritySingleNodeTestCase; import org.elasticsearch.test.SecuritySingleNodeTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
@ -245,15 +244,13 @@ public class ReloadRemoteClusterCredentialsIT extends SecuritySingleNodeTestCase
capturedHeaders.add(Map.copyOf(threadPool.getThreadContext().getHeaders())); capturedHeaders.add(Map.copyOf(threadPool.getThreadContext().getHeaders()));
channel.sendResponse( channel.sendResponse(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN),
new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN), InternalAggregations.EMPTY,
InternalAggregations.EMPTY, null,
null, false,
null, null,
false, null,
null, 1,
1
),
null, null,
1, 1,
1, 1,

View file

@ -20,7 +20,6 @@ import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.security.ScrollHelper; import org.elasticsearch.xpack.core.security.ScrollHelper;
@ -83,22 +82,28 @@ public class ScrollHelperIntegTests extends ESSingleNodeTestCase {
String scrollId = randomAlphaOfLength(5); String scrollId = randomAlphaOfLength(5);
SearchHit[] hits = new SearchHit[] { new SearchHit(1), new SearchHit(2) }; SearchHit[] hits = new SearchHit[] { new SearchHit(1), new SearchHit(2) };
InternalSearchResponse internalResponse = new InternalSearchResponse(
new SearchHits(hits, new TotalHits(3, TotalHits.Relation.EQUAL_TO), 1),
null,
null,
null,
false,
false,
1
);
Answer<?> returnResponse = invocation -> { Answer<?> returnResponse = invocation -> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[1]; ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[1];
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse(internalResponse, scrollId, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY) new SearchResponse(
new SearchHits(hits, new TotalHits(3, TotalHits.Relation.EQUAL_TO), 1),
null,
null,
false,
false,
null,
1,
scrollId,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
)
); );
return null; return null;
}; };

View file

@ -28,7 +28,6 @@ import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.search.TransportClearScrollAction;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
@ -201,15 +200,13 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
new SearchResponseSections( new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f),
new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f), null,
null, null,
null, false,
false, false,
false, null,
null, 1,
1
),
"_scrollId1", "_scrollId1",
1, 1,
1, 1,
@ -225,15 +222,13 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
(Response) new SearchResponse( (Response) new SearchResponse(
new SearchResponseSections( new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f),
new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f), null,
null, null,
null, false,
false, false,
false, null,
null, 1,
1
),
"_scrollId1", "_scrollId1",
1, 1,
1, 1,

View file

@ -60,7 +60,6 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ClusterServiceUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.MockLogAppender;
@ -411,25 +410,31 @@ public class ApiKeyServiceTests extends ESTestCase {
builder.map(buildApiKeySourceDoc("some_hash".toCharArray())); builder.map(buildApiKeySourceDoc("some_hash".toCharArray()));
searchHit.sourceRef(BytesReference.bytes(builder)); searchHit.sourceRef(BytesReference.bytes(builder));
} }
final var internalSearchResponse = new InternalSearchResponse(
new SearchHits(
new SearchHit[] { searchHit },
new TotalHits(1, TotalHits.Relation.EQUAL_TO),
randomFloat(),
null,
null,
null
),
null,
null,
null,
false,
null,
0
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse(internalSearchResponse, randomAlphaOfLengthBetween(3, 8), 1, 1, 0, 10, null, null) new SearchResponse(
new SearchHits(
new SearchHit[] { searchHit },
new TotalHits(1, TotalHits.Relation.EQUAL_TO),
randomFloat(),
null,
null,
null
),
null,
null,
false,
null,
null,
0,
randomAlphaOfLengthBetween(3, 8),
1,
1,
0,
10,
null,
null
)
); );
return null; return null;
}).when(client).search(any(SearchRequest.class), anyActionListener()); }).when(client).search(any(SearchRequest.class), anyActionListener());
@ -753,22 +758,20 @@ public class ApiKeyServiceTests extends ESTestCase {
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse( new SearchResponse(
new InternalSearchResponse( new SearchHits(
new SearchHits( searchHits.toArray(SearchHit[]::new),
searchHits.toArray(SearchHit[]::new), new TotalHits(searchHits.size(), TotalHits.Relation.EQUAL_TO),
new TotalHits(searchHits.size(), TotalHits.Relation.EQUAL_TO), randomFloat(),
randomFloat(),
null,
null,
null
),
null, null,
null, null,
null, null
false,
null,
0
), ),
null,
null,
false,
null,
null,
0,
randomAlphaOfLengthBetween(3, 8), randomAlphaOfLengthBetween(3, 8),
1, 1,
1, 1,

View file

@ -42,7 +42,6 @@ import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.XContentTestUtils; import org.elasticsearch.test.XContentTestUtils;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -270,19 +269,24 @@ public class IndexServiceAccountTokenStoreTests extends ESTestCase {
) )
) )
.toArray(SearchHit[]::new); .toArray(SearchHit[]::new);
final InternalSearchResponse internalSearchResponse;
internalSearchResponse = new InternalSearchResponse(
new SearchHits(hits, new TotalHits(nhits, TotalHits.Relation.EQUAL_TO), randomFloat(), null, null, null),
null,
null,
null,
false,
null,
0
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
l, l,
new SearchResponse(internalSearchResponse, randomAlphaOfLengthBetween(3, 8), 1, 1, 0, 10, null, null) new SearchResponse(
new SearchHits(hits, new TotalHits(nhits, TotalHits.Relation.EQUAL_TO), randomFloat(), null, null, null),
null,
null,
false,
null,
null,
0,
randomAlphaOfLengthBetween(3, 8),
1,
1,
0,
10,
null,
null
)
); );
} else if (r instanceof ClearScrollRequest) { } else if (r instanceof ClearScrollRequest) {
l.onResponse(new ClearScrollResponse(true, 1)); l.onResponse(new ClearScrollResponse(true, 1));

View file

@ -27,7 +27,6 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.mustache.MustacheScriptEngine; import org.elasticsearch.script.mustache.MustacheScriptEngine;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
@ -355,25 +354,31 @@ public class NativeRoleMappingStoreTests extends ESTestCase {
mapping.toXContent(builder, ToXContent.EMPTY_PARAMS); mapping.toXContent(builder, ToXContent.EMPTY_PARAMS);
searchHit.sourceRef(BytesReference.bytes(builder)); searchHit.sourceRef(BytesReference.bytes(builder));
} }
final var internalSearchResponse = new InternalSearchResponse(
new SearchHits(
new SearchHit[] { searchHit },
new TotalHits(1, TotalHits.Relation.EQUAL_TO),
randomFloat(),
null,
null,
null
),
null,
null,
null,
false,
null,
0
);
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse(internalSearchResponse, randomAlphaOfLengthBetween(3, 8), 1, 1, 0, 10, null, null) new SearchResponse(
new SearchHits(
new SearchHit[] { searchHit },
new TotalHits(1, TotalHits.Relation.EQUAL_TO),
randomFloat(),
null,
null,
null
),
null,
null,
false,
null,
null,
0,
randomAlphaOfLengthBetween(3, 8),
1,
1,
0,
10,
null,
null
)
); );
return null; return null;
}).when(client).search(any(SearchRequest.class), anyActionListener()); }).when(client).search(any(SearchRequest.class), anyActionListener());

View file

@ -22,7 +22,6 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
@ -822,15 +821,13 @@ public class NativePrivilegeStoreTests extends ESTestCase {
private static SearchResponse buildSearchResponse(SearchHit[] hits) { private static SearchResponse buildSearchResponse(SearchHit[] hits) {
return new SearchResponse( return new SearchResponse(
new SearchResponseSections( new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f),
new SearchHits(hits, new TotalHits(hits.length, TotalHits.Relation.EQUAL_TO), 0f), null,
null, null,
null, false,
false, false,
false, null,
null, 1,
1
),
"_scrollId1", "_scrollId1",
1, 1,
1, 1,

View file

@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -339,15 +338,13 @@ public class TimeBasedCheckpointProviderTests extends ESTestCase {
private static SearchResponse newSearchResponse(long totalHits) { private static SearchResponse newSearchResponse(long totalHits) {
return new SearchResponse( return new SearchResponse(
new SearchResponseSections( new SearchHits(SearchHits.EMPTY, new TotalHits(totalHits, TotalHits.Relation.EQUAL_TO), 0),
new SearchHits(SearchHits.EMPTY, new TotalHits(totalHits, TotalHits.Relation.EQUAL_TO), 0), null,
null, null,
null, false,
false, false,
false, null,
null, 0,
0
),
null, null,
1, 1,
1, 1,

View file

@ -31,7 +31,6 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.PointInTimeBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchContextId;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
@ -540,16 +539,14 @@ public class ClientTransformIndexerTests extends ESTestCase {
listener.onFailure(new SearchContextMissingException(new ShardSearchContextId("sc_missing", 42))); listener.onFailure(new SearchContextMissingException(new ShardSearchContextId("sc_missing", 42)));
} else { } else {
SearchResponse response = new SearchResponse( SearchResponse response = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
null, null,
1, 1,
1, 1,

View file

@ -30,7 +30,6 @@ import org.elasticsearch.index.reindex.DeleteByQueryRequest;
import org.elasticsearch.script.ScriptException; import org.elasticsearch.script.ScriptException;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -224,16 +223,14 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
void doGetInitialProgress(SearchRequest request, ActionListener<SearchResponse> responseListener) { void doGetInitialProgress(SearchRequest request, ActionListener<SearchResponse> responseListener) {
responseListener.onResponse( responseListener.onResponse(
new SearchResponse( new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,
@ -375,16 +372,14 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
null null
); );
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f),
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,
@ -513,16 +508,14 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
); );
final SearchResponse searchResponse = new SearchResponse( final SearchResponse searchResponse = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,
@ -605,16 +598,14 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
); );
final SearchResponse searchResponse = new SearchResponse( final SearchResponse searchResponse = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,
@ -700,16 +691,14 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
); );
final SearchResponse searchResponse = new SearchResponse( final SearchResponse searchResponse = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,

View file

@ -26,7 +26,6 @@ import org.elasticsearch.index.reindex.BulkByScrollTask;
import org.elasticsearch.index.reindex.DeleteByQueryRequest; import org.elasticsearch.index.reindex.DeleteByQueryRequest;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -80,16 +79,14 @@ import static org.mockito.Mockito.mock;
public class TransformIndexerStateTests extends ESTestCase { public class TransformIndexerStateTests extends ESTestCase {
private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse( private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,

View file

@ -26,7 +26,6 @@ import org.elasticsearch.index.reindex.BulkByScrollTask;
import org.elasticsearch.index.reindex.DeleteByQueryRequest; import org.elasticsearch.index.reindex.DeleteByQueryRequest;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileResults;
import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -76,16 +75,14 @@ import static org.mockito.Mockito.mock;
public class TransformIndexerTests extends ESTestCase { public class TransformIndexerTests extends ESTestCase {
private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse( private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse(
new InternalSearchResponse( new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs
// Simulate completely null aggs null,
null, new Suggest(Collections.emptyList()),
new Suggest(Collections.emptyList()), false,
new SearchProfileResults(Collections.emptyMap()), false,
false, new SearchProfileResults(Collections.emptyMap()),
false, 1,
1
),
"", "",
1, 1,
1, 1,

View file

@ -8,7 +8,6 @@
package org.elasticsearch.xpack.transform.transforms.pivot; package org.elasticsearch.xpack.transform.transforms.pivot;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.TermsQueryBuilder; import org.elasticsearch.index.query.TermsQueryBuilder;
@ -112,8 +111,22 @@ public class CompositeBucketsChangeCollectorTests extends ESTestCase {
}); });
Aggregations aggs = new Aggregations(Collections.singletonList(composite)); Aggregations aggs = new Aggregations(Collections.singletonList(composite));
SearchResponseSections sections = new SearchResponseSections(null, aggs, null, false, null, null, 1); SearchResponse response = new SearchResponse(
SearchResponse response = new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null); null,
aggs,
null,
false,
null,
null,
1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
);
try { try {
collector.processSearchResponse(response); collector.processSearchResponse(response);

View file

@ -8,7 +8,6 @@
package org.elasticsearch.xpack.transform.transforms.pivot; package org.elasticsearch.xpack.transform.transforms.pivot;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.RangeQueryBuilder; import org.elasticsearch.index.query.RangeQueryBuilder;
@ -171,16 +170,22 @@ public class DateHistogramFieldCollectorTests extends ESTestCase {
} }
private static SearchResponse buildSearchResponse(SingleValue minTimestamp, SingleValue maxTimestamp) { private static SearchResponse buildSearchResponse(SingleValue minTimestamp, SingleValue maxTimestamp) {
SearchResponseSections sections = new SearchResponseSections( return new SearchResponse(
null, null,
new Aggregations(Arrays.asList(minTimestamp, maxTimestamp)), new Aggregations(Arrays.asList(minTimestamp, maxTimestamp)),
null, null,
false, false,
null, null,
null, null,
1 1,
null,
1,
1,
0,
0,
ShardSearchFailure.EMPTY_ARRAY,
null
); );
return new SearchResponse(sections, null, 1, 1, 0, 0, ShardSearchFailure.EMPTY_ARRAY, null);
} }
} }

View file

@ -14,7 +14,6 @@ import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.aggregations.AggregationsPlugin; import org.elasticsearch.aggregations.AggregationsPlugin;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -327,9 +326,7 @@ public class PivotTests extends ESTestCase {
} }
private static SearchResponse searchResponseFromAggs(Aggregations aggs) { private static SearchResponse searchResponseFromAggs(Aggregations aggs) {
SearchResponseSections sections = new SearchResponseSections(null, aggs, null, false, null, null, 1); return new SearchResponse(null, aggs, null, false, null, null, 1, null, 10, 5, 0, 0, new ShardSearchFailure[0], null);
SearchResponse searchResponse = new SearchResponse(sections, null, 10, 5, 0, 0, new ShardSearchFailure[0], null);
return searchResponse;
} }
private class MyMockClient extends NoOpClient { private class MyMockClient extends NoOpClient {
@ -359,17 +356,14 @@ public class PivotTests extends ESTestCase {
} }
} }
final SearchResponseSections sections = new SearchResponseSections( final SearchResponse response = new SearchResponse(
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0), new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0),
null, null,
null, null,
false, false,
null, null,
null, null,
1 1,
);
final SearchResponse response = new SearchResponse(
sections,
null, null,
10, 10,
searchFailures.size() > 0 ? 0 : 5, searchFailures.size() > 0 ? 0 : 5,

View file

@ -11,7 +11,6 @@ import com.wdtinc.mapbox_vector_tile.build.MvtLayerProps;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.document.DocumentField;
import org.elasticsearch.common.geo.GeoBoundingBox; import org.elasticsearch.common.geo.GeoBoundingBox;
@ -147,21 +146,14 @@ public class RestVectorTileAction extends BaseRestHandler {
.collect(Collectors.toList()) .collect(Collectors.toList())
); );
final SearchResponse meta = new SearchResponse( final SearchResponse meta = new SearchResponse(
new SearchResponseSections( // remove actual hits
new SearchHits( new SearchHits(SearchHits.EMPTY, searchResponse.getHits().getTotalHits(), searchResponse.getHits().getMaxScore()),
SearchHits.EMPTY, aggsWithoutGridAndBounds,
searchResponse.getHits().getTotalHits(), searchResponse.getSuggest(),
searchResponse.getHits().getMaxScore() searchResponse.isTimedOut(),
), // remove actual hits searchResponse.isTerminatedEarly(),
aggsWithoutGridAndBounds, searchResponse.getProfileResults() == null ? null : new SearchProfileResults(searchResponse.getProfileResults()),
searchResponse.getSuggest(), searchResponse.getNumReducePhases(),
searchResponse.isTimedOut(),
searchResponse.isTerminatedEarly(),
searchResponse.getProfileResults() == null
? null
: new SearchProfileResults(searchResponse.getProfileResults()),
searchResponse.getNumReducePhases()
),
searchResponse.getScrollId(), searchResponse.getScrollId(),
searchResponse.getTotalShards(), searchResponse.getTotalShards(),
searchResponse.getSuccessfulShards(), searchResponse.getSuccessfulShards(),

View file

@ -16,7 +16,6 @@ import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionContext; import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionContext;
import org.elasticsearch.xpack.core.watcher.watch.Payload; import org.elasticsearch.xpack.core.watcher.watch.Payload;
@ -105,17 +104,14 @@ public class CompareConditionSearchTests extends AbstractWatcherIntegrationTestC
hit.score(1f); hit.score(1f);
hit.shard(new SearchShardTarget("a", new ShardId("a", "indexUUID", 0), null)); hit.shard(new SearchShardTarget("a", new ShardId("a", "indexUUID", 0), null));
InternalSearchResponse internalSearchResponse = new InternalSearchResponse( SearchResponse response = new SearchResponse(
new SearchHits(new SearchHit[] { hit }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1f), new SearchHits(new SearchHit[] { hit }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1f),
null, null,
null, null,
false,
false,
null, null,
false, 1,
false,
1
);
SearchResponse response = new SearchResponse(
internalSearchResponse,
"", "",
3, 3,
3, 3,

View file

@ -16,7 +16,6 @@ import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchResponseSections;
import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.search.TransportClearScrollAction;
@ -43,6 +42,7 @@ import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -172,22 +172,12 @@ public class WatcherServiceTests extends ESTestCase {
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(RefreshRequest.class), anyActionListener()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(RefreshRequest.class), anyActionListener());
// empty scroll response, no further scrolling needed
SearchResponseSections scrollSearchSections = new SearchResponseSections(
SearchHits.EMPTY_WITH_TOTAL_HITS,
null,
null,
false,
false,
null,
1
);
doAnswer(invocation -> { doAnswer(invocation -> {
ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[2]; ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[2];
// empty scroll response, no further scrolling needed
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse( SearchResponseUtils.emptyWithTotalHits(
scrollSearchSections,
"scrollId", "scrollId",
1, 1,
1, 1,
@ -223,12 +213,26 @@ public class WatcherServiceTests extends ESTestCase {
when(parser.parse(eq(id), eq(true), any(), eq(XContentType.JSON), anyLong(), anyLong())).thenReturn(watch); when(parser.parse(eq(id), eq(true), any(), eq(XContentType.JSON), anyLong(), anyLong())).thenReturn(watch);
} }
SearchHits searchHits = new SearchHits(hits, new TotalHits(count, TotalHits.Relation.EQUAL_TO), 1.0f); SearchHits searchHits = new SearchHits(hits, new TotalHits(count, TotalHits.Relation.EQUAL_TO), 1.0f);
SearchResponseSections sections = new SearchResponseSections(searchHits, null, null, false, false, null, 1);
doAnswer(invocation -> { doAnswer(invocation -> {
ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[2]; ActionListener<SearchResponse> listener = (ActionListener<SearchResponse>) invocation.getArguments()[2];
ActionListener.respondAndRelease( ActionListener.respondAndRelease(
listener, listener,
new SearchResponse(sections, "scrollId", 1, 1, 0, 10, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY) new SearchResponse(
searchHits,
null,
null,
false,
false,
null,
1,
"scrollId",
1,
1,
0,
10,
ShardSearchFailure.EMPTY_ARRAY,
SearchResponse.Clusters.EMPTY
)
); );
return null; return null;
}).when(client).execute(eq(TransportSearchAction.TYPE), any(SearchRequest.class), anyActionListener()); }).when(client).execute(eq(TransportSearchAction.TYPE), any(SearchRequest.class), anyActionListener());

View file

@ -21,7 +21,7 @@ import org.elasticsearch.script.ScriptException;
import org.elasticsearch.script.ScriptMetadata; import org.elasticsearch.script.ScriptMetadata;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptType; import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentBuilder;
@ -95,8 +95,7 @@ public class ScriptConditionTests extends ESTestCase {
public void testExecute() throws Exception { public void testExecute() throws Exception {
ScriptCondition condition = new ScriptCondition(mockScript("ctx.payload.hits.total.value > 1"), scriptService); ScriptCondition condition = new ScriptCondition(mockScript("ctx.payload.hits.total.value > 1"), scriptService);
SearchResponse response = new SearchResponse( SearchResponse response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
3, 3,
3, 3,
@ -121,8 +120,7 @@ public class ScriptConditionTests extends ESTestCase {
singletonMap("threshold", 1) singletonMap("threshold", 1)
); );
ScriptCondition executable = new ScriptCondition(script, scriptService); ScriptCondition executable = new ScriptCondition(script, scriptService);
SearchResponse response = new SearchResponse( SearchResponse response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
3, 3,
3, 3,
@ -147,8 +145,7 @@ public class ScriptConditionTests extends ESTestCase {
parser.nextToken(); parser.nextToken();
ExecutableCondition executable = ScriptCondition.parse(scriptService, "_watch", parser); ExecutableCondition executable = ScriptCondition.parse(scriptService, "_watch", parser);
SearchResponse response = new SearchResponse( SearchResponse response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
3, 3,
3, 3,
@ -223,8 +220,7 @@ public class ScriptConditionTests extends ESTestCase {
public void testScriptConditionThrowException() throws Exception { public void testScriptConditionThrowException() throws Exception {
ScriptCondition condition = new ScriptCondition(mockScript("null.foo"), scriptService); ScriptCondition condition = new ScriptCondition(mockScript("null.foo"), scriptService);
SearchResponse response = new SearchResponse( SearchResponse response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
3, 3,
3, 3,
@ -247,8 +243,7 @@ public class ScriptConditionTests extends ESTestCase {
mockScript("ctx.trigger.scheduled_time.toInstant().toEpochMill() < new Date().time"), mockScript("ctx.trigger.scheduled_time.toInstant().toEpochMill() < new Date().time"),
scriptService scriptService
); );
SearchResponse response = new SearchResponse( SearchResponse response = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
3, 3,
3, 3,

View file

@ -46,8 +46,8 @@ import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
@ -231,26 +231,8 @@ public class TriggeredWatchStoreTests extends ESTestCase {
hit.shard(new SearchShardTarget("_node_id", new ShardId(index, 0), null)); hit.shard(new SearchShardTarget("_node_id", new ShardId(index, 0), null));
hit.sourceRef(source); hit.sourceRef(source);
hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f); hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f);
SearchResponse searchResponse2 = new SearchResponse( SearchResponse searchResponse2 = new SearchResponse(hits, null, null, false, null, null, 1, "_scrollId1", 1, 1, 0, 1, null, null);
new InternalSearchResponse(hits, null, null, null, false, null, 1), SearchResponse searchResponse3 = SearchResponseUtils.emptyWithTotalHits("_scrollId2", 1, 1, 0, 1, null, null);
"_scrollId1",
1,
1,
0,
1,
null,
null
);
SearchResponse searchResponse3 = new SearchResponse(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"_scrollId2",
1,
1,
0,
1,
null,
null
);
doAnswer(invocation -> { doAnswer(invocation -> {
SearchScrollRequest request = (SearchScrollRequest) invocation.getArguments()[1]; SearchScrollRequest request = (SearchScrollRequest) invocation.getArguments()[1];

View file

@ -22,8 +22,8 @@ import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptEngine; import org.elasticsearch.script.ScriptEngine;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchResponseUtils;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.NamedXContentRegistry;
@ -91,8 +91,7 @@ public class SearchInputTests extends ESTestCase {
public void testExecute() throws Exception { public void testExecute() throws Exception {
ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class); ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class);
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
1, 1,
1, 1,
@ -132,8 +131,7 @@ public class SearchInputTests extends ESTestCase {
public void testDifferentSearchType() throws Exception { public void testDifferentSearchType() throws Exception {
ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class); ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class);
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
1, 1,
1, 1,
@ -187,8 +185,7 @@ public class SearchInputTests extends ESTestCase {
public void testThatEmptyRequestBodyWorks() throws Exception { public void testThatEmptyRequestBodyWorks() throws Exception {
ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class); ArgumentCaptor<SearchRequest> requestCaptor = ArgumentCaptor.forClass(SearchRequest.class);
PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> searchFuture = new PlainActionFuture<>();
SearchResponse searchResponse = new SearchResponse( SearchResponse searchResponse = SearchResponseUtils.emptyWithTotalHits(
InternalSearchResponse.EMPTY_WITH_TOTAL_HITS,
"", "",
1, 1,
1, 1,