From ecb01c15b9a6645f22f153eb099a377e70e398c8 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 8 Feb 2017 14:40:08 +0100 Subject: [PATCH] Fold InternalSearchHits and friends into their interfaces (#23042) We have a bunch of interfaces that have only a single implementation for 6 years now. These interfaces are pretty useless from a SW development perspective and only add unnecessary abstractions. They also require lots of casting in many places where we expect that there is only one concrete implementation. This change removes the interfaces, makes all of the classes final and removes the duplicate `foo` `getFoo` accessors in favor of `getFoo` from these classes. --- .../search/TransportNoopSearchAction.java | 8 +- .../byscroll/ClientScrollableHitSource.java | 2 +- .../action/search/SearchPhaseController.java | 20 +- ...SearchScrollQueryThenFetchAsyncAction.java | 2 - .../rest/action/cat/RestCountAction.java | 2 +- .../rest/action/document/RestCountAction.java | 2 +- .../org/elasticsearch/search/SearchHit.java | 867 ++++++++++++++-- .../elasticsearch/search/SearchHitField.java | 99 +- .../org/elasticsearch/search/SearchHits.java | 200 +++- .../{internal => }/SearchSortValues.java | 5 +- .../metrics/tophits/InternalTopHits.java | 17 +- .../metrics/tophits/TopHitsAggregator.java | 10 +- .../ExpandCollapseSearchResponseListener.java | 10 +- .../search/fetch/FetchPhase.java | 32 +- .../search/fetch/FetchSearchResult.java | 15 +- .../search/fetch/FetchSubPhase.java | 10 +- .../subphase/DocValueFieldsFetchSubPhase.java | 9 +- .../fetch/subphase/ExplainFetchSubPhase.java | 4 +- .../fetch/subphase/InnerHitsContext.java | 16 +- .../subphase/InnerHitsFetchSubPhase.java | 10 +- .../subphase/MatchedQueriesFetchSubPhase.java | 6 +- .../subphase/ParentFieldSubFetchPhase.java | 5 +- .../subphase/ScriptFieldsFetchSubPhase.java | 7 +- .../subphase/highlight/HighlightPhase.java | 8 +- .../subphase/highlight/PlainHighlighter.java | 2 +- .../highlight/PostingsHighlighter.java | 2 +- .../highlight/UnifiedHighlighter.java | 2 +- .../search/internal/InternalSearchHit.java | 943 ------------------ .../internal/InternalSearchHitField.java | 128 --- .../search/internal/InternalSearchHits.java | 228 ----- .../internal/InternalSearchResponse.java | 11 +- .../completion/CompletionSuggestion.java | 12 +- .../action/IndicesRequestIT.java | 4 +- .../admin/cluster/node/tasks/TasksIT.java | 6 +- .../action/bulk/BulkProcessorRetryIT.java | 2 +- .../AsyncBulkByScrollActionTests.java | 13 +- .../search/SearchPhaseControllerTests.java | 14 +- .../elasticsearch/aliases/IndexAliasesIT.java | 36 +- .../broadcast/BroadcastActionsIT.java | 2 +- .../bwcompat/IpFieldBwCompatIT.java | 6 +- .../OldIndexBackwardsCompatibilityIT.java | 8 +- .../bwcompat/RepositoryUpgradabilityIT.java | 6 +- .../bwcompat/RestoreBackwardsCompatIT.java | 2 +- .../cluster/MinimumMasterNodesIT.java | 4 +- .../allocation/FilteringAllocationIT.java | 6 +- .../DiscoveryWithServiceDisruptionsIT.java | 2 +- .../document/DocumentActionsIT.java | 4 +- .../index/IndexWithShadowReplicasIT.java | 2 +- .../mapper/CopyToMapperIntegrationIT.java | 4 +- .../ExternalValuesMapperIntegrationIT.java | 10 +- .../mapper/GeoPointFieldMapperTests.java | 8 +- .../mapper/MultiFieldsIntegrationIT.java | 16 +- .../TokenCountFieldMapperIntegrationIT.java | 12 +- .../query/CommonTermsQueryParserTests.java | 2 +- .../index/store/ExceptionRetryIT.java | 5 +- .../elasticsearch/indexing/IndexActionIT.java | 16 +- .../elasticsearch/indices/flush/FlushIT.java | 6 +- .../mapping/UpdateMappingIntegrationIT.java | 2 +- .../indices/recovery/IndexRecoveryIT.java | 2 +- .../template/SimpleIndexTemplateIT.java | 16 +- .../recovery/RecoveryWhileUnderLoadIT.java | 6 +- .../elasticsearch/recovery/RelocationIT.java | 16 +- .../elasticsearch/routing/AliasRoutingIT.java | 86 +- .../routing/PartitionedRoutingIT.java | 6 +- .../routing/SimpleRoutingIT.java | 30 +- .../elasticsearch/script/IndexLookupIT.java | 6 +- .../elasticsearch/script/ScriptFieldIT.java | 2 +- .../InternalSearchHitTests.java | 82 +- .../InternalSearchHitsTests.java | 22 +- ...ityTests.java => NestedIdentityTests.java} | 44 +- .../{internal => }/SearchSortValuesTests.java | 2 +- .../AggregationsIntegrationIT.java | 4 +- .../aggregations/bucket/ChildrenIT.java | 8 +- .../aggregations/bucket/DateHistogramIT.java | 2 +- .../metrics/AbstractGeoTestCase.java | 2 +- .../aggregations/metrics/TopHitsIT.java | 140 +-- .../metrics/tophits/InternalTopHitsTests.java | 30 +- .../search/basic/SearchWhileRelocatingIT.java | 4 +- .../basic/TransportTwoNodesSearchIT.java | 98 +- .../search/child/ChildQuerySearchIT.java | 664 ++++++------ .../search/fetch/FetchSubPhasePluginIT.java | 11 +- .../subphase/FetchSourceSubPhaseTests.java | 22 +- .../search/fetch/subphase/InnerHitsIT.java | 106 +- .../fetch/subphase/MatchedQueriesIT.java | 134 +-- .../highlight/HighlighterSearchIT.java | 28 +- .../search/fields/SearchFieldsIT.java | 224 ++--- .../functionscore/DecayFunctionScoreIT.java | 34 +- .../functionscore/ExplainableScriptIT.java | 6 +- .../FunctionScoreFieldValueIT.java | 2 +- .../search/functionscore/FunctionScoreIT.java | 14 +- .../functionscore/FunctionScorePluginIT.java | 2 +- .../search/functionscore/QueryRescorerIT.java | 72 +- .../functionscore/RandomScoreFunctionIT.java | 14 +- .../search/geo/GeoBoundingBoxIT.java | 32 +- .../search/geo/GeoPolygonIT.java | 8 +- .../search/geo/GeoShapeQueryTests.java | 22 +- .../search/morelikethis/MoreLikeThisIT.java | 8 +- .../search/nested/SimpleNestedIT.java | 174 ++-- .../search/preference/SearchPreferenceIT.java | 30 +- .../search/profile/query/QueryProfilerIT.java | 5 +- .../elasticsearch/search/query/ExistsIT.java | 3 +- .../search/query/MultiMatchQueryIT.java | 32 +- .../search/query/QueryStringIT.java | 4 +- .../search/query/SearchQueryIT.java | 50 +- .../search/query/SimpleQueryStringIT.java | 6 +- .../scriptfilter/ScriptQuerySearchIT.java | 36 +- .../search/scroll/DuelScrollIT.java | 18 +- .../search/scroll/SearchScrollIT.java | 92 +- .../SearchScrollWithFailingNodesIT.java | 8 +- .../search/searchafter/SearchAfterIT.java | 8 +- .../search/sort/FieldSortIT.java | 510 +++++----- .../search/sort/GeoDistanceIT.java | 124 +-- .../search/sort/SimpleSortIT.java | 40 +- .../search/source/MetadataFetchingIT.java | 8 +- .../search/source/SourceFetchingIT.java | 18 +- .../suggest/CompletionSuggestSearchIT.java | 11 +- .../similarity/SimilarityIT.java | 8 +- .../DedicatedClusterSnapshotRestoreIT.java | 18 +- .../SharedClusterSnapshotRestoreIT.java | 66 +- .../versioning/SimpleVersioningIT.java | 6 +- .../script/mustache/SearchTemplateIT.java | 8 +- .../PercolatorHighlightSubFetchPhase.java | 10 +- .../percolator/PercolatorQuerySearchIT.java | 16 +- .../reindex/DeleteByQueryBasicTests.java | 2 +- .../url/URLSnapshotRestoreTests.java | 4 +- .../azure/AzureSnapshotRestoreTests.java | 34 +- .../repositories/hdfs/HdfsTests.java | 3 +- .../s3/AbstractS3SnapshotRestoreTest.java | 47 +- .../index/store/AbstractAzureFsTestCase.java | 2 +- .../ESBlobStoreRepositoryIntegTestCase.java | 4 +- .../elasticsearch/test/ESIntegTestCase.java | 4 +- .../hamcrest/ElasticsearchAssertions.java | 22 +- 132 files changed, 2936 insertions(+), 3398 deletions(-) rename core/src/main/java/org/elasticsearch/search/{internal => }/SearchSortValues.java (97%) delete mode 100644 core/src/main/java/org/elasticsearch/search/internal/InternalSearchHit.java delete mode 100644 core/src/main/java/org/elasticsearch/search/internal/InternalSearchHitField.java delete mode 100644 core/src/main/java/org/elasticsearch/search/internal/InternalSearchHits.java rename core/src/test/java/org/elasticsearch/search/{internal => }/InternalSearchHitTests.java (71%) rename core/src/test/java/org/elasticsearch/search/{internal => }/InternalSearchHitsTests.java (81%) rename core/src/test/java/org/elasticsearch/search/{internal/InternalNestedIdentityTests.java => NestedIdentityTests.java} (69%) rename core/src/test/java/org/elasticsearch/search/{internal => }/SearchSortValuesTests.java (99%) diff --git a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java index c4397684bc4..1e09e890a0b 100644 --- a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java +++ b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/search/TransportNoopSearchAction.java @@ -28,8 +28,8 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.profile.SearchProfileShardResults; import org.elasticsearch.search.suggest.Suggest; @@ -49,8 +49,8 @@ public class TransportNoopSearchAction extends HandledTransportAction listener) { listener.onResponse(new SearchResponse(new InternalSearchResponse( - new InternalSearchHits( - new InternalSearchHit[0], 0L, 0.0f), + new SearchHits( + new SearchHit[0], 0L, 0.0f), new InternalAggregations(Collections.emptyList()), new Suggest(Collections.emptyList()), new SearchProfileShardResults(Collections.emptyMap()), false, false), "", 1, 1, 0, new ShardSearchFailure[0])); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/byscroll/ClientScrollableHitSource.java b/core/src/main/java/org/elasticsearch/action/bulk/byscroll/ClientScrollableHitSource.java index 4f2aefc1011..9fc02e29e62 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/byscroll/ClientScrollableHitSource.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/byscroll/ClientScrollableHitSource.java @@ -255,7 +255,7 @@ public class ClientScrollableHitSource extends ScrollableHitSource { private T fieldValue(String fieldName) { SearchHitField field = delegate.field(fieldName); - return field == null ? null : field.value(); + return field == null ? null : field.getValue(); } } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 33c8859e044..54451556032 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -45,8 +45,8 @@ import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; @@ -386,7 +386,7 @@ public class SearchPhaseController extends AbstractComponent { return InternalSearchResponse.empty(); } List> fetchResults = fetchResultsArr.asList(); - InternalSearchHits hits = getHits(reducedQueryPhase, ignoreFrom, sortedDocs, fetchResultsArr); + SearchHits hits = getHits(reducedQueryPhase, ignoreFrom, sortedDocs, fetchResultsArr); if (reducedQueryPhase.suggest != null) { if (!fetchResults.isEmpty()) { int currentOffset = hits.getHits().length; @@ -401,7 +401,7 @@ public class SearchPhaseController extends AbstractComponent { FetchSearchResult fetchResult = searchResultProvider.fetchResult(); int fetchResultIndex = fetchResult.counterGetAndIncrement(); if (fetchResultIndex < fetchResult.hits().internalHits().length) { - InternalSearchHit hit = fetchResult.hits().internalHits()[fetchResultIndex]; + SearchHit hit = fetchResult.hits().internalHits()[fetchResultIndex]; CompletionSuggestion.Entry.Option suggestOption = suggestionOptions.get(scoreDocIndex - currentOffset); hit.score(shardDoc.score); @@ -417,8 +417,8 @@ public class SearchPhaseController extends AbstractComponent { return reducedQueryPhase.buildResponse(hits); } - private InternalSearchHits getHits(ReducedQueryPhase reducedQueryPhase, boolean ignoreFrom, ScoreDoc[] sortedDocs, - AtomicArray fetchResultsArr) { + private SearchHits getHits(ReducedQueryPhase reducedQueryPhase, boolean ignoreFrom, ScoreDoc[] sortedDocs, + AtomicArray fetchResultsArr) { List> fetchResults = fetchResultsArr.asList(); boolean sorted = false; int sortScoreIndex = -1; @@ -445,7 +445,7 @@ public class SearchPhaseController extends AbstractComponent { // with collapsing we can have more fetch hits than sorted docs numSearchHits = Math.min(sortedDocs.length, numSearchHits); // merge hits - List hits = new ArrayList<>(); + List hits = new ArrayList<>(); if (!fetchResults.isEmpty()) { for (int i = 0; i < numSearchHits; i++) { ScoreDoc shardDoc = sortedDocs[i]; @@ -456,7 +456,7 @@ public class SearchPhaseController extends AbstractComponent { FetchSearchResult fetchResult = fetchResultProvider.fetchResult(); int index = fetchResult.counterGetAndIncrement(); if (index < fetchResult.hits().internalHits().length) { - InternalSearchHit searchHit = fetchResult.hits().internalHits()[index]; + SearchHit searchHit = fetchResult.hits().internalHits()[index]; searchHit.score(shardDoc.score); searchHit.shard(fetchResult.shardTarget()); if (sorted) { @@ -470,7 +470,7 @@ public class SearchPhaseController extends AbstractComponent { } } } - return new InternalSearchHits(hits.toArray(new InternalSearchHit[hits.size()]), reducedQueryPhase.totalHits, + return new SearchHits(hits.toArray(new SearchHit[hits.size()]), reducedQueryPhase.totalHits, reducedQueryPhase.maxScore); } @@ -596,7 +596,7 @@ public class SearchPhaseController extends AbstractComponent { * Creates a new search response from the given merged hits. * @see #merge(boolean, ScoreDoc[], ReducedQueryPhase, AtomicArray) */ - public InternalSearchResponse buildResponse(InternalSearchHits hits) { + public InternalSearchResponse buildResponse(SearchHits hits) { return new InternalSearchResponse(hits, aggregations, suggest, shardResults, timedOut, terminatedEarly); } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchScrollQueryThenFetchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchScrollQueryThenFetchAsyncAction.java index 6d8d3a42be3..13c81c1d5e6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchScrollQueryThenFetchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchScrollQueryThenFetchAsyncAction.java @@ -30,14 +30,12 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.fetch.QueryFetchSearchResult; import org.elasticsearch.search.fetch.ShardFetchRequest; import org.elasticsearch.search.internal.InternalScrollSearchRequest; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.ScrollQuerySearchResult; -import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestCountAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestCountAction.java index 00eb89a9c02..bbc7ffa250c 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestCountAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestCountAction.java @@ -91,7 +91,7 @@ public class RestCountAction extends AbstractCatAction { private Table buildTable(RestRequest request, SearchResponse response) { Table table = getTableWithHeader(request); table.startRow(); - table.addCell(response.getHits().totalHits()); + table.addCell(response.getHits().getTotalHits()); table.endRow(); return table; diff --git a/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java b/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java index 5de436b38c3..004fab81b47 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/document/RestCountAction.java @@ -91,7 +91,7 @@ public class RestCountAction extends BaseRestHandler { if (terminateAfter != DEFAULT_TERMINATE_AFTER) { builder.field("terminated_early", response.isTerminatedEarly()); } - builder.field("count", response.getHits().totalHits()); + builder.field("count", response.getHits().getTotalHits()); buildBroadcastShardsHeader(builder, request, response.getTotalShards(), response.getSuccessfulShards(), response.getFailedShards(), response.getShardFailures()); diff --git a/core/src/main/java/org/elasticsearch/search/SearchHit.java b/core/src/main/java/org/elasticsearch/search/SearchHit.java index ef9aef8fbb7..8f8d1a98ed3 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/core/src/main/java/org/elasticsearch/search/SearchHit.java @@ -21,210 +21,889 @@ package org.elasticsearch.search; import org.apache.lucene.search.Explanation; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.compress.CompressorFactory; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; +import org.elasticsearch.search.lookup.SourceLookup; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.Objects; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +import static java.util.Collections.unmodifiableMap; +import static org.elasticsearch.common.lucene.Lucene.readExplanation; +import static org.elasticsearch.common.lucene.Lucene.writeExplanation; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.common.xcontent.XContentParserUtils.parseStoredFieldsValue; +import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; +import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownToken; +import static org.elasticsearch.search.fetch.subphase.highlight.HighlightField.readHighlightField; /** * A single search hit. * * @see SearchHits */ -public interface SearchHit extends Streamable, ToXContentObject, Iterable { +public final class SearchHit implements Streamable, ToXContentObject, Iterable { + + private transient int docId; + + private static final float DEFAULT_SCORE = Float.NEGATIVE_INFINITY; + private float score = DEFAULT_SCORE; + + private Text id; + private Text type; + + private NestedIdentity nestedIdentity; + + private long version = -1; + + private BytesReference source; + + private Map fields = emptyMap(); + + private Map highlightFields = null; + + private SearchSortValues sortValues = SearchSortValues.EMPTY; + + private String[] matchedQueries = Strings.EMPTY_ARRAY; + + private Explanation explanation; + + @Nullable + private SearchShardTarget shard; + + private transient String index; + + private Map sourceAsMap; + private byte[] sourceAsBytes; + + private Map innerHits; + + private SearchHit() { + + } + + public SearchHit(int docId) { + this(docId, null, null, null); + } + + public SearchHit(int docId, String id, Text type, Map fields) { + this(docId, id, type, null, fields); + } + + public SearchHit(int nestedTopDocId, String id, Text type, NestedIdentity nestedIdentity, Map fields) { + this.docId = nestedTopDocId; + if (id != null) { + this.id = new Text(id); + } else { + this.id = null; + } + this.type = type; + this.nestedIdentity = nestedIdentity; + this.fields = fields; + } + + public int docId() { + return this.docId; + } + + public void score(float score) { + this.score = score; + } /** * The score. */ - float score(); + public float getScore() { + return this.score; + } + + public void version(long version) { + this.version = version; + } /** - * The score. + * The version of the hit. */ - float getScore(); + public long getVersion() { + return this.version; + } /** * The index of the hit. */ - String index(); - - /** - * The index of the hit. - */ - String getIndex(); + public String getIndex() { + return this.index; + } /** * The id of the document. */ - String id(); - - /** - * The id of the document. - */ - String getId(); + public String getId() { + return id != null ? id.string() : null; + } /** * The type of the document. */ - String type(); - - /** - * The type of the document. - */ - String getType(); + public String getType() { + return type != null ? type.string() : null; + } /** * If this is a nested hit then nested reference information is returned otherwise null is returned. */ - NestedIdentity getNestedIdentity(); - - /** - * The version of the hit. - */ - long version(); - - /** - * The version of the hit. - */ - long getVersion(); + public NestedIdentity getNestedIdentity() { + return nestedIdentity; + } /** * Returns bytes reference, also un compress the source if needed. */ - BytesReference sourceRef(); + public BytesReference getSourceRef() { + if (this.source == null) { + return null; + } + + try { + this.source = CompressorFactory.uncompressIfNeeded(this.source); + return this.source; + } catch (IOException e) { + throw new ElasticsearchParseException("failed to decompress source", e); + } + } /** - * Returns bytes reference, also un compress the source if needed. + * Sets representation, might be compressed.... */ - BytesReference getSourceRef(); - - /** - * The source of the document (can be null). Note, its a copy of the source - * into a byte array, consider using {@link #sourceRef()} so there won't be a need to copy. - */ - byte[] source(); + public SearchHit sourceRef(BytesReference source) { + this.source = source; + this.sourceAsBytes = null; + this.sourceAsMap = null; + return this; + } /** * Is the source available or not. A source with no fields will return true. This will return false if {@code fields} doesn't contain * {@code _source} or if source is disabled in the mapping. */ - boolean hasSource(); - - /** - * The source of the document as a map (can be null). - */ - Map getSource(); + public boolean hasSource() { + return source != null; + } /** * The source of the document as string (can be null). */ - String sourceAsString(); + public String getSourceAsString() { + if (source == null) { + return null; + } + try { + return XContentHelper.convertToJson(getSourceRef(), false); + } catch (IOException e) { + throw new ElasticsearchParseException("failed to convert source to a json string"); + } + } - /** - * The source of the document as string (can be null). - */ - String getSourceAsString(); /** * The source of the document as a map (can be null). */ - Map sourceAsMap() throws ElasticsearchParseException; + public Map getSourceAsMap() { + if (source == null) { + return null; + } + if (sourceAsMap != null) { + return sourceAsMap; + } - /** - * If enabled, the explanation of the search hit. - */ - Explanation explanation(); + sourceAsMap = SourceLookup.sourceAsMap(source); + return sourceAsMap; + } - /** - * If enabled, the explanation of the search hit. - */ - Explanation getExplanation(); + @Override + public Iterator iterator() { + return fields.values().iterator(); + } /** * The hit field matching the given field name. */ - SearchHitField field(String fieldName); + public SearchHitField field(String fieldName) { + return getFields().get(fieldName); + } /** * A map of hit fields (from field name to hit fields) if additional fields * were required to be loaded. */ - Map fields(); + public Map getFields() { + return fields == null ? emptyMap() : fields; + } - /** - * A map of hit fields (from field name to hit fields) if additional fields - * were required to be loaded. - */ - Map getFields(); + // returns the fields without handling null cases + public Map fieldsOrNull() { + return fields; + } + + public void fields(Map fields) { + this.fields = fields; + } /** * A map of highlighted fields. */ - Map highlightFields(); + public Map getHighlightFields() { + return highlightFields == null ? emptyMap() : highlightFields; + } - /** - * A map of highlighted fields. - */ - Map getHighlightFields(); + public void highlightFields(Map highlightFields) { + this.highlightFields = highlightFields; + } + + public void sortValues(Object[] sortValues, DocValueFormat[] sortValueFormats) { + sortValues(new SearchSortValues(sortValues, sortValueFormats)); + } + + public void sortValues(SearchSortValues sortValues) { + this.sortValues = sortValues; + } /** * An array of the sort values used. */ - Object[] sortValues(); + public Object[] getSortValues() { + return sortValues.sortValues(); + } /** - * An array of the sort values used. + * If enabled, the explanation of the search hit. */ - Object[] getSortValues(); + public Explanation getExplanation() { + return explanation; + } - /** - * The set of query and filter names the query matched with. Mainly makes sense for compound filters and queries. - */ - String[] matchedQueries(); - - /** - * The set of query and filter names the query matched with. Mainly makes sense for compound filters and queries. - */ - String[] getMatchedQueries(); + public void explanation(Explanation explanation) { + this.explanation = explanation; + } /** * The shard of the search hit. */ - SearchShardTarget shard(); + public SearchShardTarget getShard() { + return shard; + } + + public void shard(SearchShardTarget target) { + this.shard = target; + if (target != null) { + this.index = target.getIndex(); + } + } + + public void matchedQueries(String[] matchedQueries) { + this.matchedQueries = matchedQueries; + } /** - * The shard of the search hit. + * The set of query and filter names the query matched with. Mainly makes sense for compound filters and queries. */ - SearchShardTarget getShard(); + public String[] getMatchedQueries() { + return this.matchedQueries; + } /** * @return Inner hits or null if there are none */ - Map getInnerHits(); + @SuppressWarnings("unchecked") + public Map getInnerHits() { + return (Map) innerHits; + } + + public void setInnerHits(Map innerHits) { + this.innerHits = innerHits; + } + + public static class Fields { + static final String _INDEX = "_index"; + static final String _TYPE = "_type"; + static final String _ID = "_id"; + static final String _VERSION = "_version"; + static final String _SCORE = "_score"; + static final String FIELDS = "fields"; + static final String HIGHLIGHT = "highlight"; + static final String SORT = "sort"; + static final String MATCHED_QUERIES = "matched_queries"; + static final String _EXPLANATION = "_explanation"; + static final String VALUE = "value"; + static final String DESCRIPTION = "description"; + static final String DETAILS = "details"; + static final String INNER_HITS = "inner_hits"; + static final String _SHARD = "_shard"; + static final String _NODE = "_node"; + } + + // public because we render hit as part of completion suggestion option + public XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException { + List metaFields = new ArrayList<>(); + List otherFields = new ArrayList<>(); + if (fields != null && !fields.isEmpty()) { + for (SearchHitField field : fields.values()) { + if (field.getValues().isEmpty()) { + continue; + } + if (field.isMetadataField()) { + metaFields.add(field); + } else { + otherFields.add(field); + } + } + } + + // For inner_hit hits shard is null and that is ok, because the parent search hit has all this information. + // Even if this was included in the inner_hit hits this would be the same, so better leave it out. + if (getExplanation() != null && shard != null) { + builder.field(Fields._SHARD, shard.getShardId()); + builder.field(Fields._NODE, shard.getNodeIdText()); + } + if (nestedIdentity != null) { + nestedIdentity.toXContent(builder, params); + } else { + if (index != null) { + builder.field(Fields._INDEX, index); + } + if (type != null) { + builder.field(Fields._TYPE, type); + } + if (id != null) { + builder.field(Fields._ID, id); + } + } + if (version != -1) { + builder.field(Fields._VERSION, version); + } + if (Float.isNaN(score)) { + builder.nullField(Fields._SCORE); + } else { + builder.field(Fields._SCORE, score); + } + for (SearchHitField field : metaFields) { + Object value = field.getValue(); + builder.field(field.getName(), value); + } + if (source != null) { + XContentHelper.writeRawField(SourceFieldMapper.NAME, source, builder, params); + } + if (!otherFields.isEmpty()) { + builder.startObject(Fields.FIELDS); + for (SearchHitField field : otherFields) { + builder.startArray(field.getName()); + for (Object value : field.getValues()) { + builder.value(value); + } + builder.endArray(); + } + builder.endObject(); + } + if (highlightFields != null && !highlightFields.isEmpty()) { + builder.startObject(Fields.HIGHLIGHT); + for (HighlightField field : highlightFields.values()) { + field.toXContent(builder, params); + } + builder.endObject(); + } + sortValues.toXContent(builder, params); + if (matchedQueries.length > 0) { + builder.startArray(Fields.MATCHED_QUERIES); + for (String matchedFilter : matchedQueries) { + builder.value(matchedFilter); + } + builder.endArray(); + } + if (getExplanation() != null) { + builder.field(Fields._EXPLANATION); + buildExplanation(builder, getExplanation()); + } + if (innerHits != null) { + builder.startObject(Fields.INNER_HITS); + for (Map.Entry entry : innerHits.entrySet()) { + builder.startObject(entry.getKey()); + entry.getValue().toXContent(builder, params); + builder.endObject(); + } + builder.endObject(); + } + return builder; + } + + public static SearchHit fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + String currentFieldName = null; + String type = null, id = null; + String index = null; + float score = DEFAULT_SCORE; + long version = -1; + SearchSortValues sortValues = SearchSortValues.EMPTY; + NestedIdentity nestedIdentity = null; + Map highlightFields = new HashMap<>(); + BytesReference parsedSource = null; + List matchedQueries = new ArrayList<>(); + Map fields = new HashMap<>(); + Explanation explanation = null; + ShardId shardId = null; + String nodeId = null; + Map innerHits = null; + while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if (Fields._TYPE.equals(currentFieldName)) { + type = parser.text(); + } else if (Fields._INDEX.equals(currentFieldName)) { + index = parser.text(); + } else if (Fields._ID.equals(currentFieldName)) { + id = parser.text(); + } else if (Fields._SCORE.equals(currentFieldName)) { + score = parser.floatValue(); + } else if (Fields._VERSION.equals(currentFieldName)) { + version = parser.longValue(); + } else if (Fields._SHARD.equals(currentFieldName)) { + shardId = ShardId.fromString(parser.text()); + } else if (Fields._NODE.equals(currentFieldName)) { + nodeId = parser.text(); + } else if (MapperService.isMetadataField(currentFieldName)) { + List values = new ArrayList<>(); + values.add(parseStoredFieldsValue(parser)); + fields.put(currentFieldName, new SearchHitField(currentFieldName, values)); + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.VALUE_NULL) { + if (Fields._SCORE.equals(currentFieldName)) { + score = Float.NaN; + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (SourceFieldMapper.NAME.equals(currentFieldName)) { + try (XContentBuilder builder = XContentBuilder.builder(parser.contentType().xContent())) { + //the original document gets slightly modified: whitespaces or pretty printing are not preserved, + //it all depends on the current builder settings + builder.copyCurrentStructure(parser); + parsedSource = builder.bytes(); + } + } else if (Fields.HIGHLIGHT.equals(currentFieldName)) { + while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + HighlightField highlightField = HighlightField.fromXContent(parser); + highlightFields.put(highlightField.getName(), highlightField); + } + } else if (Fields.FIELDS.equals(currentFieldName)) { + while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + String fieldName = parser.currentName(); + List values = new ArrayList<>(); + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.nextToken(), parser::getTokenLocation); + while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + values.add(parseStoredFieldsValue(parser)); + } + fields.put(fieldName, new SearchHitField(fieldName, values)); + } + } else if (Fields._EXPLANATION.equals(currentFieldName)) { + explanation = parseExplanation(parser); + } else if (Fields.INNER_HITS.equals(currentFieldName)) { + innerHits = new HashMap<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + // parse the key + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser::getTokenLocation); + String name = parser.currentName(); + innerHits.put(name, SearchHits.fromXContent(parser)); + parser.nextToken(); + ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.currentToken(), parser::getTokenLocation); + } + } else if (NestedIdentity.Fields._NESTED.equals(currentFieldName)) { + nestedIdentity = NestedIdentity.fromXContent(parser); + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (Fields.SORT.equals(currentFieldName)) { + sortValues = SearchSortValues.fromXContent(parser); + } else if (Fields.MATCHED_QUERIES.equals(currentFieldName)) { + while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + matchedQueries.add(parser.text()); + } + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else { + throwUnknownToken(token, parser.getTokenLocation()); + } + } + SearchHit searchHit = new SearchHit(-1, id, new Text(type), nestedIdentity, Collections.emptyMap()); + searchHit.index = index; + searchHit.score(score); + searchHit.version(version); + searchHit.sortValues(sortValues); + searchHit.highlightFields(highlightFields); + searchHit.sourceRef(parsedSource); + searchHit.explanation(explanation); + searchHit.setInnerHits(innerHits); + if (matchedQueries.size() > 0) { + searchHit.matchedQueries(matchedQueries.toArray(new String[matchedQueries.size()])); + } + if (shardId != null && nodeId != null) { + searchHit.shard(new SearchShardTarget(nodeId, shardId)); + } + searchHit.fields(fields); + return searchHit; + } + + private static Explanation parseExplanation(XContentParser parser) throws IOException { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + XContentParser.Token token; + Float value = null; + String description = null; + List details = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, () -> parser.getTokenLocation()); + String currentFieldName = parser.currentName(); + token = parser.nextToken(); + if (Fields.VALUE.equals(currentFieldName)) { + value = parser.floatValue(); + } else if (Fields.DESCRIPTION.equals(currentFieldName)) { + description = parser.textOrNull(); + } else if (Fields.DETAILS.equals(currentFieldName)) { + ensureExpectedToken(XContentParser.Token.START_ARRAY, token, () -> parser.getTokenLocation()); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + details.add(parseExplanation(parser)); + } + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } + if (value == null) { + throw new ParsingException(parser.getTokenLocation(), "missing explanation value"); + } + if (description == null) { + throw new ParsingException(parser.getTokenLocation(), "missing explanation description"); + } + return Explanation.match(value, description, details); + } + + private void buildExplanation(XContentBuilder builder, Explanation explanation) throws IOException { + builder.startObject(); + builder.field(Fields.VALUE, explanation.getValue()); + builder.field(Fields.DESCRIPTION, explanation.getDescription()); + Explanation[] innerExps = explanation.getDetails(); + if (innerExps != null) { + builder.startArray(Fields.DETAILS); + for (Explanation exp : innerExps) { + buildExplanation(builder, exp); + } + builder.endArray(); + } + builder.endObject(); + + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + toInnerXContent(builder, params); + builder.endObject(); + return builder; + } + + public static SearchHit readSearchHit(StreamInput in) throws IOException { + SearchHit hit = new SearchHit(); + hit.readFrom(in); + return hit; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + score = in.readFloat(); + id = in.readOptionalText(); + type = in.readOptionalText(); + nestedIdentity = in.readOptionalWriteable(NestedIdentity::new); + version = in.readLong(); + source = in.readBytesReference(); + if (source.length() == 0) { + source = null; + } + if (in.readBoolean()) { + explanation = readExplanation(in); + } + int size = in.readVInt(); + if (size == 0) { + fields = emptyMap(); + } else if (size == 1) { + SearchHitField hitField = SearchHitField.readSearchHitField(in); + fields = singletonMap(hitField.getName(), hitField); + } else { + Map fields = new HashMap<>(); + for (int i = 0; i < size; i++) { + SearchHitField hitField = SearchHitField.readSearchHitField(in); + fields.put(hitField.getName(), hitField); + } + this.fields = unmodifiableMap(fields); + } + + size = in.readVInt(); + if (size == 0) { + highlightFields = emptyMap(); + } else if (size == 1) { + HighlightField field = readHighlightField(in); + highlightFields = singletonMap(field.name(), field); + } else { + Map highlightFields = new HashMap<>(); + for (int i = 0; i < size; i++) { + HighlightField field = readHighlightField(in); + highlightFields.put(field.name(), field); + } + this.highlightFields = unmodifiableMap(highlightFields); + } + + sortValues = new SearchSortValues(in); + + size = in.readVInt(); + if (size > 0) { + matchedQueries = new String[size]; + for (int i = 0; i < size; i++) { + matchedQueries[i] = in.readString(); + } + } + // we call the setter here because that also sets the local index parameter + shard(in.readOptionalWriteable(SearchShardTarget::new)); + size = in.readVInt(); + if (size > 0) { + innerHits = new HashMap<>(size); + for (int i = 0; i < size; i++) { + String key = in.readString(); + SearchHits value = SearchHits.readSearchHits(in); + innerHits.put(key, value); + } + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeFloat(score); + out.writeOptionalText(id); + out.writeOptionalText(type); + out.writeOptionalWriteable(nestedIdentity); + out.writeLong(version); + out.writeBytesReference(source); + if (explanation == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + writeExplanation(out, explanation); + } + if (fields == null) { + out.writeVInt(0); + } else { + out.writeVInt(fields.size()); + for (SearchHitField hitField : getFields().values()) { + hitField.writeTo(out); + } + } + if (highlightFields == null) { + out.writeVInt(0); + } else { + out.writeVInt(highlightFields.size()); + for (HighlightField highlightField : highlightFields.values()) { + highlightField.writeTo(out); + } + } + sortValues.writeTo(out); + + if (matchedQueries.length == 0) { + out.writeVInt(0); + } else { + out.writeVInt(matchedQueries.length); + for (String matchedFilter : matchedQueries) { + out.writeString(matchedFilter); + } + } + out.writeOptionalWriteable(shard); + if (innerHits == null) { + out.writeVInt(0); + } else { + out.writeVInt(innerHits.size()); + for (Map.Entry entry : innerHits.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SearchHit other = (SearchHit) obj; + return Objects.equals(id, other.id) + && Objects.equals(type, other.type) + && Objects.equals(nestedIdentity, other.nestedIdentity) + && Objects.equals(version, other.version) + && Objects.equals(source, other.source) + && Objects.equals(fields, other.fields) + && Objects.equals(getHighlightFields(), other.getHighlightFields()) + && Arrays.equals(matchedQueries, other.matchedQueries) + && Objects.equals(explanation, other.explanation) + && Objects.equals(shard, other.shard) + && Objects.equals(innerHits, other.innerHits); + } + + @Override + public int hashCode() { + return Objects.hash(id, type, nestedIdentity, version, source, fields, getHighlightFields(), Arrays.hashCode(matchedQueries), + explanation, shard, innerHits); + } /** * Encapsulates the nested identity of a hit. */ - interface NestedIdentity { + public static final class NestedIdentity implements Writeable, ToXContent { + + private Text field; + private int offset; + private NestedIdentity child; + + public NestedIdentity(String field, int offset, NestedIdentity child) { + this.field = new Text(field); + this.offset = offset; + this.child = child; + } + + NestedIdentity(StreamInput in) throws IOException { + field = in.readOptionalText(); + offset = in.readInt(); + child = in.readOptionalWriteable(NestedIdentity::new); + } /** * Returns the nested field in the source this hit originates from */ - Text getField(); + public Text getField() { + return field; + } /** * Returns the offset in the nested array of objects in the source this hit */ - int getOffset(); + public int getOffset() { + return offset; + } /** * Returns the next child nested level if there is any, otherwise null is returned. * * In the case of mappings with multiple levels of nested object fields */ - NestedIdentity getChild(); + public NestedIdentity getChild() { + return child; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalText(field); + out.writeInt(offset); + out.writeOptionalWriteable(child); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(Fields._NESTED); + return innerToXContent(builder, params); + } + + /** + * Rendering of the inner XContent object without the leading field name. This way the structure innerToXContent renders and + * fromXContent parses correspond to each other. + */ + XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (field != null) { + builder.field(Fields._NESTED_FIELD, field); + } + if (offset != -1) { + builder.field(Fields._NESTED_OFFSET, offset); + } + if (child != null) { + builder = child.toXContent(builder, params); + } + builder.endObject(); + return builder; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "nested_identity", + ctorArgs -> new NestedIdentity((String) ctorArgs[0], (int) ctorArgs[1], (NestedIdentity) ctorArgs[2])); + static { + PARSER.declareString(constructorArg(), new ParseField(Fields._NESTED_FIELD)); + PARSER.declareInt(constructorArg(), new ParseField(Fields._NESTED_OFFSET)); + PARSER.declareObject(optionalConstructorArg(), PARSER, new ParseField(Fields._NESTED)); + } + + public static NestedIdentity fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + NestedIdentity other = (NestedIdentity) obj; + return Objects.equals(field, other.field) && + Objects.equals(offset, other.offset) && + Objects.equals(child, other.child); + } + + @Override + public int hashCode() { + return Objects.hash(field, offset, child); + } + + public static class Fields { + static final String _NESTED = "_nested"; + static final String _NESTED_FIELD = "field"; + static final String _NESTED_OFFSET = "offset"; + } } } diff --git a/core/src/main/java/org/elasticsearch/search/SearchHitField.java b/core/src/main/java/org/elasticsearch/search/SearchHitField.java index 5747bbebef8..fbb211b90d8 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchHitField.java +++ b/core/src/main/java/org/elasticsearch/search/SearchHitField.java @@ -19,49 +19,108 @@ package org.elasticsearch.search; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.index.mapper.MapperService; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; +import java.util.Objects; /** * A single field name and values part of a {@link SearchHit}. * * @see SearchHit */ -public interface SearchHitField extends Streamable, Iterable { +public final class SearchHitField implements Streamable, Iterable { + + private String name; + private List values; + + private SearchHitField() { + } + + public SearchHitField(String name, List values) { + this.name = name; + this.values = values; + } /** * The name of the field. */ - String name(); - - /** - * The name of the field. - */ - String getName(); + public String getName() { + return name; + } /** * The first value of the hit. */ - V value(); - - /** - * The first value of the hit. - */ - V getValue(); + public V getValue() { + if (values == null || values.isEmpty()) { + return null; + } + return (V)values.get(0); + } /** * The field values. */ - List values(); - - /** - * The field values. - */ - List getValues(); + public List getValues() { + return values; + } /** * @return The field is a metadata field */ - boolean isMetadataField(); + public boolean isMetadataField() { + return MapperService.isMetadataField(name); + } + + @Override + public Iterator iterator() { + return values.iterator(); + } + + public static SearchHitField readSearchHitField(StreamInput in) throws IOException { + SearchHitField result = new SearchHitField(); + result.readFrom(in); + return result; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + name = in.readString(); + int size = in.readVInt(); + values = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + values.add(in.readGenericValue()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeVInt(values.size()); + for (Object value : values) { + out.writeGenericValue(value); + } + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SearchHitField other = (SearchHitField) obj; + return Objects.equals(name, other.name) + && Objects.equals(values, other.values); + } + + @Override + public int hashCode() { + return Objects.hash(name, values); + } } diff --git a/core/src/main/java/org/elasticsearch/search/SearchHits.java b/core/src/main/java/org/elasticsearch/search/SearchHits.java index 400e2ebc44f..ada09d2e38b 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchHits.java +++ b/core/src/main/java/org/elasticsearch/search/SearchHits.java @@ -19,48 +19,202 @@ package org.elasticsearch.search; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; -/** - * The hits of a search request. - * - * - */ -public interface SearchHits extends Streamable, ToXContent, Iterable { +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; + +public final class SearchHits implements Streamable, ToXContent, Iterable { + + public static SearchHits empty() { + // We shouldn't use static final instance, since that could directly be returned by native transport clients + return new SearchHits(EMPTY, 0, 0); + } + + public static final SearchHit[] EMPTY = new SearchHit[0]; + + private SearchHit[] hits; + + public long totalHits; + + private float maxScore; + + SearchHits() { + + } + + public SearchHits(SearchHit[] hits, long totalHits, float maxScore) { + this.hits = hits; + this.totalHits = totalHits; + this.maxScore = maxScore; + } + + public void shardTarget(SearchShardTarget shardTarget) { + for (SearchHit hit : hits) { + hit.shard(shardTarget); + } + } /** * The total number of hits that matches the search request. */ - long totalHits(); + public long getTotalHits() { + return totalHits; + } - /** - * The total number of hits that matches the search request. - */ - long getTotalHits(); /** * The maximum score of this query. */ - float maxScore(); - - /** - * The maximum score of this query. - */ - float getMaxScore(); + public float getMaxScore() { + return maxScore; + } /** * The hits of the search request (based on the search type, and from / size provided). */ - SearchHit[] hits(); + public SearchHit[] getHits() { + return this.hits; + } /** * Return the hit as the provided position. */ - SearchHit getAt(int position); + public SearchHit getAt(int position) { + return hits[position]; + } - /** - * The hits of the search request (based on the search type, and from / size provided). - */ - SearchHit[] getHits(); + @Override + public Iterator iterator() { + return Arrays.stream(getHits()).iterator(); + } + + public SearchHit[] internalHits() { + return this.hits; + } + + static final class Fields { + static final String HITS = "hits"; + static final String TOTAL = "total"; + static final String MAX_SCORE = "max_score"; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.HITS); + builder.field(Fields.TOTAL, totalHits); + if (Float.isNaN(maxScore)) { + builder.nullField(Fields.MAX_SCORE); + } else { + builder.field(Fields.MAX_SCORE, maxScore); + } + builder.field(Fields.HITS); + builder.startArray(); + for (SearchHit hit : hits) { + hit.toXContent(builder, params); + } + builder.endArray(); + builder.endObject(); + return builder; + } + + public static SearchHits fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() != XContentParser.Token.START_OBJECT) { + parser.nextToken(); + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); + } + XContentParser.Token token = parser.currentToken(); + String currentFieldName = null; + List hits = new ArrayList<>(); + long totalHits = 0; + float maxScore = 0f; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if (Fields.TOTAL.equals(currentFieldName)) { + totalHits = parser.longValue(); + } else if (Fields.MAX_SCORE.equals(currentFieldName)) { + maxScore = parser.floatValue(); + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.VALUE_NULL) { + if (Fields.MAX_SCORE.equals(currentFieldName)) { + maxScore = Float.NaN; // NaN gets rendered as null-field + } else { + throwUnknownField(currentFieldName, parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + hits.add(SearchHit.fromXContent(parser)); + } + } + } + SearchHits searchHits = new SearchHits(hits.toArray(new SearchHit[hits.size()]), totalHits, + maxScore); + return searchHits; + } + + + public static SearchHits readSearchHits(StreamInput in) throws IOException { + SearchHits hits = new SearchHits(); + hits.readFrom(in); + return hits; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + totalHits = in.readVLong(); + maxScore = in.readFloat(); + int size = in.readVInt(); + if (size == 0) { + hits = EMPTY; + } else { + hits = new SearchHit[size]; + for (int i = 0; i < hits.length; i++) { + hits[i] = SearchHit.readSearchHit(in); + } + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(totalHits); + out.writeFloat(maxScore); + out.writeVInt(hits.length); + if (hits.length > 0) { + for (SearchHit hit : hits) { + hit.writeTo(out); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SearchHits other = (SearchHits) obj; + return Objects.equals(totalHits, other.totalHits) + && Objects.equals(maxScore, other.maxScore) + && Arrays.equals(hits, other.hits); + } + + @Override + public int hashCode() { + return Objects.hash(totalHits, maxScore, Arrays.hashCode(hits)); + } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/SearchSortValues.java b/core/src/main/java/org/elasticsearch/search/SearchSortValues.java similarity index 97% rename from core/src/main/java/org/elasticsearch/search/internal/SearchSortValues.java rename to core/src/main/java/org/elasticsearch/search/SearchSortValues.java index 9aa29d7768f..d3d55ff481a 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/SearchSortValues.java +++ b/core/src/main/java/org/elasticsearch/search/SearchSortValues.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.internal; +package org.elasticsearch.search; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.io.stream.StreamInput; @@ -27,8 +27,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParserUtils; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.internal.InternalSearchHit.Fields; +import org.elasticsearch.search.SearchHit.Fields; import java.io.IOException; import java.util.Arrays; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java index 3f712f99b35..e60bf94dbcd 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java @@ -32,8 +32,7 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; import java.io.IOException; import java.util.Arrays; @@ -47,9 +46,9 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi private int from; private int size; private TopDocs topDocs; - private InternalSearchHits searchHits; + private SearchHits searchHits; - public InternalTopHits(String name, int from, int size, TopDocs topDocs, InternalSearchHits searchHits, + public InternalTopHits(String name, int from, int size, TopDocs topDocs, SearchHits searchHits, List pipelineAggregators, Map metaData) { super(name, pipelineAggregators, metaData); this.from = from; @@ -67,7 +66,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi size = in.readVInt(); topDocs = Lucene.readTopDocs(in); assert topDocs != null; - searchHits = InternalSearchHits.readSearchHits(in); + searchHits = SearchHits.readSearchHits(in); } @Override @@ -98,7 +97,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi @Override public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { - InternalSearchHits[] shardHits = new InternalSearchHits[aggregations.size()]; + SearchHits[] shardHits = new SearchHits[aggregations.size()]; final TopDocs reducedTopDocs; final TopDocs[] shardDocs; @@ -124,16 +123,16 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi } final int[] tracker = new int[shardHits.length]; - InternalSearchHit[] hits = new InternalSearchHit[reducedTopDocs.scoreDocs.length]; + SearchHit[] hits = new SearchHit[reducedTopDocs.scoreDocs.length]; for (int i = 0; i < reducedTopDocs.scoreDocs.length; i++) { ScoreDoc scoreDoc = reducedTopDocs.scoreDocs[i]; int position; do { position = tracker[scoreDoc.shardIndex]++; } while (shardDocs[scoreDoc.shardIndex].scoreDocs[position] != scoreDoc); - hits[i] = (InternalSearchHit) shardHits[scoreDoc.shardIndex].getAt(position); + hits[i] = shardHits[scoreDoc.shardIndex].getAt(position); } - return new InternalTopHits(name, from, size, reducedTopDocs, new InternalSearchHits(hits, reducedTopDocs.totalHits, + return new InternalTopHits(name, from, size, reducedTopDocs, new SearchHits(hits, reducedTopDocs.totalHits, reducedTopDocs.getMaxScore()), pipelineAggregators(), getMetaData()); } catch (IOException e) { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java index 92041747730..6b8e4413357 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java @@ -41,8 +41,8 @@ import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SubSearchContext; import org.elasticsearch.search.rescore.RescoreSearchContext; @@ -165,10 +165,10 @@ public class TopHitsAggregator extends MetricsAggregator { subSearchContext.docIdsToLoad(docIdsToLoad, 0, docIdsToLoad.length); fetchPhase.execute(subSearchContext); FetchSearchResult fetchResult = subSearchContext.fetchResult(); - InternalSearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits(); + SearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits(); for (int i = 0; i < internalHits.length; i++) { ScoreDoc scoreDoc = topDocs.scoreDocs[i]; - InternalSearchHit searchHitFields = internalHits[i]; + SearchHit searchHitFields = internalHits[i]; searchHitFields.shard(subSearchContext.shardTarget()); searchHitFields.score(scoreDoc.score); if (scoreDoc instanceof FieldDoc) { @@ -190,7 +190,7 @@ public class TopHitsAggregator extends MetricsAggregator { } else { topDocs = Lucene.EMPTY_TOP_DOCS; } - return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, InternalSearchHits.empty(), + return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, SearchHits.empty(), pipelineAggregators(), metaData()); } diff --git a/core/src/main/java/org/elasticsearch/search/collapse/ExpandCollapseSearchResponseListener.java b/core/src/main/java/org/elasticsearch/search/collapse/ExpandCollapseSearchResponseListener.java index 9a45b7a9fec..b9caa5216cb 100644 --- a/core/src/main/java/org/elasticsearch/search/collapse/ExpandCollapseSearchResponseListener.java +++ b/core/src/main/java/org/elasticsearch/search/collapse/ExpandCollapseSearchResponseListener.java @@ -29,7 +29,6 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.internal.InternalSearchHit; import java.util.HashMap; import java.util.Objects; @@ -56,9 +55,8 @@ public class ExpandCollapseSearchResponseListener implements BiConsumer(1)); + if (hit.getInnerHits() == null) { + hit.setInnerHits(new HashMap<>(1)); } - internalHit.getInnerHits().put(collapseBuilder.getInnerHit().getName(), innerHits); + hit.getInnerHits().put(collapseBuilder.getInnerHit().getName(), innerHits); } } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index dcf55872e35..83af0b9abd4 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -46,9 +46,7 @@ import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchPhase; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.search.fetch.subphase.InnerHitsFetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHitField; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SourceLookup; import org.elasticsearch.tasks.TaskCancelledException; @@ -134,7 +132,7 @@ public class FetchPhase implements SearchPhase { } } - InternalSearchHit[] hits = new InternalSearchHit[context.docIdsToLoadSize()]; + SearchHit[] hits = new SearchHit[context.docIdsToLoadSize()]; FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext(); for (int index = 0; index < context.docIdsToLoadSize(); index++) { if(context.isCancelled()) { @@ -145,7 +143,7 @@ public class FetchPhase implements SearchPhase { LeafReaderContext subReaderContext = context.searcher().getIndexReader().leaves().get(readerIndex); int subDocId = docId - subReaderContext.docBase; - final InternalSearchHit searchHit; + final SearchHit searchHit; try { int rootDocId = findRootDocumentIfNested(context, subReaderContext, subDocId); if (rootDocId != -1) { @@ -168,7 +166,7 @@ public class FetchPhase implements SearchPhase { fetchSubPhase.hitsExecute(context, hits); } - context.fetchResult().hits(new InternalSearchHits(hits, context.queryResult().topDocs().totalHits, context.queryResult().topDocs().getMaxScore())); + context.fetchResult().hits(new SearchHits(hits, context.queryResult().topDocs().totalHits, context.queryResult().topDocs().getMaxScore())); } private int findRootDocumentIfNested(SearchContext context, LeafReaderContext subReaderContext, int subDocId) throws IOException { @@ -181,9 +179,9 @@ public class FetchPhase implements SearchPhase { return -1; } - private InternalSearchHit createSearchHit(SearchContext context, FieldsVisitor fieldsVisitor, int docId, int subDocId, LeafReaderContext subReaderContext) { + private SearchHit createSearchHit(SearchContext context, FieldsVisitor fieldsVisitor, int docId, int subDocId, LeafReaderContext subReaderContext) { if (fieldsVisitor == null) { - return new InternalSearchHit(docId); + return new SearchHit(docId); } loadStoredFields(context, subReaderContext, fieldsVisitor, subDocId); fieldsVisitor.postProcess(context.mapperService()); @@ -192,7 +190,7 @@ public class FetchPhase implements SearchPhase { if (!fieldsVisitor.fields().isEmpty()) { searchFields = new HashMap<>(fieldsVisitor.fields().size()); for (Map.Entry> entry : fieldsVisitor.fields().entrySet()) { - searchFields.put(entry.getKey(), new InternalSearchHitField(entry.getKey(), entry.getValue())); + searchFields.put(entry.getKey(), new SearchHitField(entry.getKey(), entry.getValue())); } } @@ -203,7 +201,7 @@ public class FetchPhase implements SearchPhase { } else { typeText = documentMapper.typeText(); } - InternalSearchHit searchHit = new InternalSearchHit(docId, fieldsVisitor.uid().id(), typeText, searchFields); + SearchHit searchHit = new SearchHit(docId, fieldsVisitor.uid().id(), typeText, searchFields); // Set _source if requested. SourceLookup sourceLookup = context.lookup().source(); sourceLookup.setSegmentAndDocument(subReaderContext, subDocId); @@ -213,7 +211,7 @@ public class FetchPhase implements SearchPhase { return searchHit; } - private InternalSearchHit createNestedSearchHit(SearchContext context, int nestedTopDocId, int nestedSubDocId, int rootSubDocId, Set fieldNames, List fieldNamePatterns, LeafReaderContext subReaderContext) throws IOException { + private SearchHit createNestedSearchHit(SearchContext context, int nestedTopDocId, int nestedSubDocId, int rootSubDocId, Set fieldNames, List fieldNamePatterns, LeafReaderContext subReaderContext) throws IOException { // Also if highlighting is requested on nested documents we need to fetch the _source from the root document, // otherwise highlighting will attempt to fetch the _source from the nested doc, which will fail, // because the entire _source is only stored with the root document. @@ -228,7 +226,7 @@ public class FetchPhase implements SearchPhase { ObjectMapper nestedObjectMapper = documentMapper.findNestedObjectMapper(nestedSubDocId, context, subReaderContext); assert nestedObjectMapper != null; - InternalSearchHit.InternalNestedIdentity nestedIdentity = getInternalNestedIdentity(context, nestedSubDocId, subReaderContext, documentMapper, nestedObjectMapper); + SearchHit.NestedIdentity nestedIdentity = getInternalNestedIdentity(context, nestedSubDocId, subReaderContext, documentMapper, nestedObjectMapper); BytesReference source = rootFieldsVisitor.source(); if (source != null) { @@ -271,7 +269,7 @@ public class FetchPhase implements SearchPhase { context.lookup().source().setSourceContentType(contentType); } - return new InternalSearchHit(nestedTopDocId, rootFieldsVisitor.uid().id(), documentMapper.typeText(), nestedIdentity, searchFields); + return new SearchHit(nestedTopDocId, rootFieldsVisitor.uid().id(), documentMapper.typeText(), nestedIdentity, searchFields); } private Map getSearchFields(SearchContext context, int nestedSubDocId, Set fieldNames, List fieldNamePatterns, LeafReaderContext subReaderContext) { @@ -285,7 +283,7 @@ public class FetchPhase implements SearchPhase { if (!nestedFieldsVisitor.fields().isEmpty()) { searchFields = new HashMap<>(nestedFieldsVisitor.fields().size()); for (Map.Entry> entry : nestedFieldsVisitor.fields().entrySet()) { - searchFields.put(entry.getKey(), new InternalSearchHitField(entry.getKey(), entry.getValue())); + searchFields.put(entry.getKey(), new SearchHitField(entry.getKey(), entry.getValue())); } } } @@ -293,12 +291,12 @@ public class FetchPhase implements SearchPhase { return searchFields; } - private InternalSearchHit.InternalNestedIdentity getInternalNestedIdentity(SearchContext context, int nestedSubDocId, LeafReaderContext subReaderContext, DocumentMapper documentMapper, ObjectMapper nestedObjectMapper) throws IOException { + private SearchHit.NestedIdentity getInternalNestedIdentity(SearchContext context, int nestedSubDocId, LeafReaderContext subReaderContext, DocumentMapper documentMapper, ObjectMapper nestedObjectMapper) throws IOException { int currentParent = nestedSubDocId; ObjectMapper nestedParentObjectMapper; ObjectMapper current = nestedObjectMapper; String originalName = nestedObjectMapper.name(); - InternalSearchHit.InternalNestedIdentity nestedIdentity = null; + SearchHit.NestedIdentity nestedIdentity = null; do { Query parentFilter; nestedParentObjectMapper = documentMapper.findParentObjectMapper(current); @@ -335,7 +333,7 @@ public class FetchPhase implements SearchPhase { currentParent = nextParent; current = nestedObjectMapper = nestedParentObjectMapper; int currentPrefix = current == null ? 0 : current.name().length() + 1; - nestedIdentity = new InternalSearchHit.InternalNestedIdentity(originalName.substring(currentPrefix), offset, nestedIdentity); + nestedIdentity = new SearchHit.NestedIdentity(originalName.substring(currentPrefix), offset, nestedIdentity); if (current != null) { originalName = current.name(); } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/FetchSearchResult.java b/core/src/main/java/org/elasticsearch/search/fetch/FetchSearchResult.java index 6e844a5d9df..1e2def8cc61 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/FetchSearchResult.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/FetchSearchResult.java @@ -23,10 +23,9 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; -import org.elasticsearch.transport.TransportResponse; import java.io.IOException; @@ -34,7 +33,7 @@ public class FetchSearchResult extends QuerySearchResultProvider { private long id; private SearchShardTarget shardTarget; - private InternalSearchHits hits; + private SearchHits hits; // client side counter private transient int counter; @@ -72,19 +71,19 @@ public class FetchSearchResult extends QuerySearchResultProvider { this.shardTarget = shardTarget; } - public void hits(InternalSearchHits hits) { + public void hits(SearchHits hits) { assert assertNoSearchTarget(hits); this.hits = hits; } - private boolean assertNoSearchTarget(InternalSearchHits hits) { - for (SearchHit hit : hits.hits()) { + private boolean assertNoSearchTarget(SearchHits hits) { + for (SearchHit hit : hits.getHits()) { assert hit.getShard() == null : "expected null but got: " + hit.getShard(); } return true; } - public InternalSearchHits hits() { + public SearchHits hits() { return hits; } @@ -107,7 +106,7 @@ public class FetchSearchResult extends QuerySearchResultProvider { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); id = in.readLong(); - hits = InternalSearchHits.readSearchHits(in); + hits = SearchHits.readSearchHits(in); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/fetch/FetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/FetchSubPhase.java index 1783652d120..6f34eba2129 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/FetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/FetchSubPhase.java @@ -22,7 +22,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.IndexSearcher; -import org.elasticsearch.search.internal.InternalSearchHit; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.internal.SearchContext; import java.util.HashMap; @@ -34,20 +34,20 @@ import java.util.Map; public interface FetchSubPhase { class HitContext { - private InternalSearchHit hit; + private SearchHit hit; private IndexSearcher searcher; private LeafReaderContext readerContext; private int docId; private Map cache; - public void reset(InternalSearchHit hit, LeafReaderContext context, int docId, IndexSearcher searcher) { + public void reset(SearchHit hit, LeafReaderContext context, int docId, IndexSearcher searcher) { this.hit = hit; this.readerContext = context; this.docId = docId; this.searcher = searcher; } - public InternalSearchHit hit() { + public SearchHit hit() { return hit; } @@ -82,5 +82,5 @@ public interface FetchSubPhase { default void hitExecute(SearchContext context, HitContext hitContext) {} - default void hitsExecute(SearchContext context, InternalSearchHit[] hits) {} + default void hitsExecute(SearchContext context, SearchHit[] hits) {} } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/DocValueFieldsFetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/DocValueFieldsFetchSubPhase.java index d6f3318a8b7..031f7a62400 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/DocValueFieldsFetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/DocValueFieldsFetchSubPhase.java @@ -23,7 +23,6 @@ import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHitField; import org.elasticsearch.search.internal.SearchContext; import java.util.ArrayList; @@ -55,10 +54,10 @@ public final class DocValueFieldsFetchSubPhase implements FetchSubPhase { if (hitContext.hit().fieldsOrNull() == null) { hitContext.hit().fields(new HashMap<>(2)); } - SearchHitField hitField = hitContext.hit().fields().get(field); + SearchHitField hitField = hitContext.hit().getFields().get(field); if (hitField == null) { - hitField = new InternalSearchHitField(field, new ArrayList<>(2)); - hitContext.hit().fields().put(field, hitField); + hitField = new SearchHitField(field, new ArrayList<>(2)); + hitContext.hit().getFields().put(field, hitField); } MappedFieldType fieldType = context.mapperService().fullName(field); if (fieldType != null) { @@ -67,7 +66,7 @@ public final class DocValueFieldsFetchSubPhase implements FetchSubPhase { AtomicFieldData data = context.fieldData().getForField(fieldType).load(hitContext.readerContext()); ScriptDocValues values = data.getScriptValues(); values.setNextDocId(hitContext.docId()); - hitField.values().addAll(values); + hitField.getValues().addAll(values); } } } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java index 4ad7db0c757..5aabaf644e9 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java @@ -46,8 +46,8 @@ public final class ExplainFetchSubPhase implements FetchSubPhase { // we use the top level doc id, since we work with the top level searcher hitContext.hit().explanation(explanation); } catch (IOException e) { - throw new FetchPhaseExecutionException(context, "Failed to explain doc [" + hitContext.hit().type() + "#" - + hitContext.hit().id() + "]", e); + throw new FetchPhaseExecutionException(context, "Failed to explain doc [" + hitContext.hit().getType() + "#" + + hitContext.hit().getId() + "]", e); } finally { context.clearReleasables(SearchContext.Lifetime.COLLECTION); } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java index d3de22f7203..5c76328610d 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java @@ -50,7 +50,7 @@ import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHit; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SubSearchContext; @@ -287,10 +287,10 @@ public final class InnerHitsContext { public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException { final Query hitQuery; if (isParentHit(hitContext.hit())) { - String field = ParentFieldMapper.joinField(hitContext.hit().type()); - hitQuery = new DocValuesTermsQuery(field, hitContext.hit().id()); + String field = ParentFieldMapper.joinField(hitContext.hit().getType()); + hitQuery = new DocValuesTermsQuery(field, hitContext.hit().getId()); } else if (isChildHit(hitContext.hit())) { - DocumentMapper hitDocumentMapper = mapperService.documentMapper(hitContext.hit().type()); + DocumentMapper hitDocumentMapper = mapperService.documentMapper(hitContext.hit().getType()); final String parentType = hitDocumentMapper.parentFieldMapper().type(); SearchHitField parentField = hitContext.hit().field(ParentFieldMapper.NAME); if (parentField == null) { @@ -328,12 +328,12 @@ public final class InnerHitsContext { } } - private boolean isParentHit(InternalSearchHit hit) { - return hit.type().equals(documentMapper.parentFieldMapper().type()); + private boolean isParentHit(SearchHit hit) { + return hit.getType().equals(documentMapper.parentFieldMapper().type()); } - private boolean isChildHit(InternalSearchHit hit) { - DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.type()); + private boolean isChildHit(SearchHit hit) { + DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType()); return documentMapper.type().equals(hitDocumentMapper.parentFieldMapper().type()); } } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsFetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsFetchSubPhase.java index 23c63bc7eef..48294bd82c5 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsFetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsFetchSubPhase.java @@ -26,8 +26,8 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -47,7 +47,7 @@ public final class InnerHitsFetchSubPhase implements FetchSubPhase { if ((context.innerHits() != null && context.innerHits().getInnerHits().size() > 0) == false) { return; } - Map results = new HashMap<>(); + Map results = new HashMap<>(); for (Map.Entry entry : context.innerHits().getInnerHits().entrySet()) { InnerHitsContext.BaseInnerHits innerHits = entry.getValue(); TopDocs topDocs; @@ -64,10 +64,10 @@ public final class InnerHitsFetchSubPhase implements FetchSubPhase { innerHits.docIdsToLoad(docIdsToLoad, 0, docIdsToLoad.length); fetchPhase.execute(innerHits); FetchSearchResult fetchResult = innerHits.fetchResult(); - InternalSearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits(); + SearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits(); for (int i = 0; i < internalHits.length; i++) { ScoreDoc scoreDoc = topDocs.scoreDocs[i]; - InternalSearchHit searchHitFields = internalHits[i]; + SearchHit searchHitFields = internalHits[i]; searchHitFields.score(scoreDoc.score); if (scoreDoc instanceof FieldDoc) { FieldDoc fieldDoc = (FieldDoc) scoreDoc; diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesFetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesFetchSubPhase.java index 18eacffdc7c..e38090ee4d8 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesFetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesFetchSubPhase.java @@ -28,7 +28,7 @@ import org.apache.lucene.util.Bits; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHit; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext.Lifetime; @@ -42,7 +42,7 @@ import java.util.Map; public final class MatchedQueriesFetchSubPhase implements FetchSubPhase { @Override - public void hitsExecute(SearchContext context, InternalSearchHit[] hits) { + public void hitsExecute(SearchContext context, SearchHit[] hits) { if (hits.length == 0 || // in case the request has only suggest, parsed query is null context.parsedQuery() == null) { @@ -71,7 +71,7 @@ public final class MatchedQueriesFetchSubPhase implements FetchSubPhase { Bits matchingDocs = null; final IndexReader indexReader = context.searcher().getIndexReader(); for (int i = 0; i < hits.length; ++i) { - InternalSearchHit hit = hits[i]; + SearchHit hit = hits[i]; int hitReaderIndex = ReaderUtil.subIndex(hit.docId(), indexReader.leaves()); if (readerIndex != hitReaderIndex) { readerIndex = hitReaderIndex; diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ParentFieldSubFetchPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ParentFieldSubFetchPhase.java index 5cd19096beb..00b96867026 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ParentFieldSubFetchPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ParentFieldSubFetchPhase.java @@ -26,7 +26,6 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.index.mapper.ParentFieldMapper; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHitField; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -41,7 +40,7 @@ public final class ParentFieldSubFetchPhase implements FetchSubPhase { if (context.storedFieldsContext() != null && context.storedFieldsContext().fetchFields() == false) { return ; } - ParentFieldMapper parentFieldMapper = context.mapperService().documentMapper(hitContext.hit().type()).parentFieldMapper(); + ParentFieldMapper parentFieldMapper = context.mapperService().documentMapper(hitContext.hit().getType()).parentFieldMapper(); if (parentFieldMapper.active() == false) { return; } @@ -57,7 +56,7 @@ public final class ParentFieldSubFetchPhase implements FetchSubPhase { fields = new HashMap<>(); hitContext.hit().fields(fields); } - fields.put(ParentFieldMapper.NAME, new InternalSearchHitField(ParentFieldMapper.NAME, Collections.singletonList(parentId))); + fields.put(ParentFieldMapper.NAME, new SearchHitField(ParentFieldMapper.NAME, Collections.singletonList(parentId))); } public static String getParentId(ParentFieldMapper fieldMapper, LeafReader reader, int docId) { diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ScriptFieldsFetchSubPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ScriptFieldsFetchSubPhase.java index 93415f53506..c272ab6dbf0 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/ScriptFieldsFetchSubPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/ScriptFieldsFetchSubPhase.java @@ -21,7 +21,6 @@ package org.elasticsearch.search.fetch.subphase; import org.elasticsearch.script.LeafSearchScript; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHitField; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -63,7 +62,7 @@ public final class ScriptFieldsFetchSubPhase implements FetchSubPhase { hitContext.hit().fields(new HashMap<>(2)); } - SearchHitField hitField = hitContext.hit().fields().get(scriptField.name()); + SearchHitField hitField = hitContext.hit().getFields().get(scriptField.name()); if (hitField == null) { final List values; if (value instanceof Collection) { @@ -72,8 +71,8 @@ public final class ScriptFieldsFetchSubPhase implements FetchSubPhase { } else { values = Collections.singletonList(value); } - hitField = new InternalSearchHitField(scriptField.name(), values); - hitContext.hit().fields().put(scriptField.name(), hitField); + hitField = new SearchHitField(scriptField.name(), values); + hitContext.hit().getFields().put(scriptField.name(), hitField); } } } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightPhase.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightPhase.java index e89d87e4fbf..701b981e0f0 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightPhase.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightPhase.java @@ -57,17 +57,17 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase { for (SearchContextHighlight.Field field : context.highlight().fields()) { Collection fieldNamesToHighlight; if (Regex.isSimpleMatchPattern(field.field())) { - DocumentMapper documentMapper = context.mapperService().documentMapper(hitContext.hit().type()); + DocumentMapper documentMapper = context.mapperService().documentMapper(hitContext.hit().getType()); fieldNamesToHighlight = documentMapper.mappers().simpleMatchToFullName(field.field()); } else { fieldNamesToHighlight = Collections.singletonList(field.field()); } if (context.highlight().forceSource(field)) { - SourceFieldMapper sourceFieldMapper = context.mapperService().documentMapper(hitContext.hit().type()).sourceMapper(); + SourceFieldMapper sourceFieldMapper = context.mapperService().documentMapper(hitContext.hit().getType()).sourceMapper(); if (!sourceFieldMapper.enabled()) { throw new IllegalArgumentException("source is forced for fields " + fieldNamesToHighlight - + " but type [" + hitContext.hit().type() + "] has disabled _source"); + + " but type [" + hitContext.hit().getType() + "] has disabled _source"); } } @@ -129,7 +129,7 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase { } private FieldMapper getMapperForField(String fieldName, SearchContext searchContext, HitContext hitContext) { - DocumentMapper documentMapper = searchContext.mapperService().documentMapper(hitContext.hit().type()); + DocumentMapper documentMapper = searchContext.mapperService().documentMapper(hitContext.hit().getType()); // TODO: no need to lookup the doc mapper with unambiguous field names? just look at the mapper service return documentMapper.mappers().smartNameFieldMapper(fieldName); } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java index 127a008f9cc..deb1464b703 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java @@ -101,7 +101,7 @@ public class PlainHighlighter implements Highlighter { int numberOfFragments = field.fieldOptions().numberOfFragments() == 0 ? 1 : field.fieldOptions().numberOfFragments(); ArrayList fragsList = new ArrayList<>(); List textsToHighlight; - Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().type()).mappers().indexAnalyzer(); + Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().getType()).mappers().indexAnalyzer(); try { textsToHighlight = HighlightUtils.loadFieldValues(field, mapper, context, hitContext); diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PostingsHighlighter.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PostingsHighlighter.java index 330fb908cc5..34997912feb 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PostingsHighlighter.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PostingsHighlighter.java @@ -78,7 +78,7 @@ public class PostingsHighlighter implements Highlighter { List snippets = new ArrayList<>(); int numberOfFragments; try { - Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().type()).mappers().indexAnalyzer(); + Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().getType()).mappers().indexAnalyzer(); List fieldValues = HighlightUtils.loadFieldValues(field, fieldMapper, context, hitContext); CustomPostingsHighlighter highlighter; if (field.fieldOptions().numberOfFragments() == 0) { diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java index 4afce9700e5..2ffa1a28efd 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java @@ -78,7 +78,7 @@ public class UnifiedHighlighter implements Highlighter { int numberOfFragments; try { Analyzer analyzer = - context.mapperService().documentMapper(hitContext.hit().type()).mappers().indexAnalyzer(); + context.mapperService().documentMapper(hitContext.hit().getType()).mappers().indexAnalyzer(); List fieldValues = HighlightUtils.loadFieldValues(field, fieldMapper, context, hitContext); fieldValues = fieldValues.stream().map(obj -> { if (obj instanceof BytesRef) { diff --git a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHit.java b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHit.java deleted file mode 100644 index 3487312b727..00000000000 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHit.java +++ /dev/null @@ -1,943 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.internal; - -import org.apache.lucene.search.Explanation; -import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressorFactory; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.text.Text; -import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHitField; -import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; -import org.elasticsearch.search.lookup.SourceLookup; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; -import static java.util.Collections.unmodifiableMap; -import static org.elasticsearch.common.lucene.Lucene.readExplanation; -import static org.elasticsearch.common.lucene.Lucene.writeExplanation; -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; -import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; -import static org.elasticsearch.common.xcontent.XContentParserUtils.parseStoredFieldsValue; -import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; -import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownToken; -import static org.elasticsearch.search.fetch.subphase.highlight.HighlightField.readHighlightField; -import static org.elasticsearch.search.internal.InternalSearchHitField.readSearchHitField; - -public class InternalSearchHit implements SearchHit { - - private transient int docId; - - private static final float DEFAULT_SCORE = Float.NEGATIVE_INFINITY; - private float score = DEFAULT_SCORE; - - private Text id; - private Text type; - - private InternalNestedIdentity nestedIdentity; - - private long version = -1; - - private BytesReference source; - - private Map fields = emptyMap(); - - private Map highlightFields = null; - - private SearchSortValues sortValues = SearchSortValues.EMPTY; - - private String[] matchedQueries = Strings.EMPTY_ARRAY; - - private Explanation explanation; - - @Nullable - private SearchShardTarget shard; - - private transient String index; - - private Map sourceAsMap; - private byte[] sourceAsBytes; - - private Map innerHits; - - private InternalSearchHit() { - - } - - public InternalSearchHit(int docId) { - this(docId, null, null, null); - } - - public InternalSearchHit(int docId, String id, Text type, Map fields) { - this(docId, id, type, null, fields); - } - - public InternalSearchHit(int nestedTopDocId, String id, Text type, InternalNestedIdentity nestedIdentity, Map fields) { - this.docId = nestedTopDocId; - if (id != null) { - this.id = new Text(id); - } else { - this.id = null; - } - this.type = type; - this.nestedIdentity = nestedIdentity; - this.fields = fields; - } - - public int docId() { - return this.docId; - } - - public void score(float score) { - this.score = score; - } - - @Override - public float score() { - return this.score; - } - - @Override - public float getScore() { - return score(); - } - - public void version(long version) { - this.version = version; - } - - @Override - public long version() { - return this.version; - } - - @Override - public long getVersion() { - return this.version; - } - - @Override - public String index() { - return this.index; - } - - @Override - public String getIndex() { - return index(); - } - - @Override - public String id() { - return id != null ? id.string() : null; - } - - @Override - public String getId() { - return id(); - } - - @Override - public String type() { - return type != null ? type.string() : null; - } - - @Override - public String getType() { - return type(); - } - - @Override - public NestedIdentity getNestedIdentity() { - return nestedIdentity; - } - - /** - * Returns bytes reference, also un compress the source if needed. - */ - @Override - public BytesReference sourceRef() { - if (this.source == null) { - return null; - } - - try { - this.source = CompressorFactory.uncompressIfNeeded(this.source); - return this.source; - } catch (IOException e) { - throw new ElasticsearchParseException("failed to decompress source", e); - } - } - - /** - * Sets representation, might be compressed.... - */ - public InternalSearchHit sourceRef(BytesReference source) { - this.source = source; - this.sourceAsBytes = null; - this.sourceAsMap = null; - return this; - } - - @Override - public BytesReference getSourceRef() { - return sourceRef(); - } - - @Override - public byte[] source() { - if (source == null) { - return null; - } - if (sourceAsBytes != null) { - return sourceAsBytes; - } - this.sourceAsBytes = BytesReference.toBytes(sourceRef()); - return this.sourceAsBytes; - } - - @Override - public boolean hasSource() { - return source != null; - } - - @Override - public Map getSource() { - return sourceAsMap(); - } - - @Override - public String sourceAsString() { - if (source == null) { - return null; - } - try { - return XContentHelper.convertToJson(sourceRef(), false); - } catch (IOException e) { - throw new ElasticsearchParseException("failed to convert source to a json string"); - } - } - - @Override - public String getSourceAsString() { - return sourceAsString(); - } - - @Override - public Map sourceAsMap() throws ElasticsearchParseException { - if (source == null) { - return null; - } - if (sourceAsMap != null) { - return sourceAsMap; - } - - sourceAsMap = SourceLookup.sourceAsMap(source); - return sourceAsMap; - } - - @Override - public Iterator iterator() { - return fields.values().iterator(); - } - - @Override - public SearchHitField field(String fieldName) { - return fields().get(fieldName); - } - - @Override - public Map fields() { - return fields == null ? emptyMap() : fields; - } - - // returns the fields without handling null cases - public Map fieldsOrNull() { - return fields; - } - - @Override - public Map getFields() { - return fields(); - } - - public void fields(Map fields) { - this.fields = fields; - } - - @Override - public Map highlightFields() { - return highlightFields == null ? emptyMap() : highlightFields; - } - - @Override - public Map getHighlightFields() { - return highlightFields(); - } - - public void highlightFields(Map highlightFields) { - this.highlightFields = highlightFields; - } - - public void sortValues(Object[] sortValues, DocValueFormat[] sortValueFormats) { - sortValues(new SearchSortValues(sortValues, sortValueFormats)); - } - - public void sortValues(SearchSortValues sortValues) { - this.sortValues = sortValues; - } - - @Override - public Object[] sortValues() { - return sortValues.sortValues(); - } - - @Override - public Object[] getSortValues() { - return sortValues(); - } - - @Override - public Explanation explanation() { - return explanation; - } - - @Override - public Explanation getExplanation() { - return explanation(); - } - - public void explanation(Explanation explanation) { - this.explanation = explanation; - } - - @Override - public SearchShardTarget shard() { - return shard; - } - - @Override - public SearchShardTarget getShard() { - return shard(); - } - - public void shard(SearchShardTarget target) { - this.shard = target; - if (target != null) { - this.index = target.getIndex(); - } - } - - public void matchedQueries(String[] matchedQueries) { - this.matchedQueries = matchedQueries; - } - - @Override - public String[] matchedQueries() { - return this.matchedQueries; - } - - @Override - public String[] getMatchedQueries() { - return this.matchedQueries; - } - - @Override - @SuppressWarnings("unchecked") - public Map getInnerHits() { - return (Map) innerHits; - } - - public void setInnerHits(Map innerHits) { - this.innerHits = innerHits; - } - - public static class Fields { - static final String _INDEX = "_index"; - static final String _TYPE = "_type"; - static final String _ID = "_id"; - static final String _VERSION = "_version"; - static final String _SCORE = "_score"; - static final String FIELDS = "fields"; - static final String HIGHLIGHT = "highlight"; - static final String SORT = "sort"; - static final String MATCHED_QUERIES = "matched_queries"; - static final String _EXPLANATION = "_explanation"; - static final String VALUE = "value"; - static final String DESCRIPTION = "description"; - static final String DETAILS = "details"; - static final String INNER_HITS = "inner_hits"; - static final String _SHARD = "_shard"; - static final String _NODE = "_node"; - } - - // public because we render hit as part of completion suggestion option - public XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException { - List metaFields = new ArrayList<>(); - List otherFields = new ArrayList<>(); - if (fields != null && !fields.isEmpty()) { - for (SearchHitField field : fields.values()) { - if (field.values().isEmpty()) { - continue; - } - if (field.isMetadataField()) { - metaFields.add(field); - } else { - otherFields.add(field); - } - } - } - - // For inner_hit hits shard is null and that is ok, because the parent search hit has all this information. - // Even if this was included in the inner_hit hits this would be the same, so better leave it out. - if (explanation() != null && shard != null) { - builder.field(Fields._SHARD, shard.getShardId()); - builder.field(Fields._NODE, shard.getNodeIdText()); - } - if (nestedIdentity != null) { - nestedIdentity.toXContent(builder, params); - } else { - if (index != null) { - builder.field(Fields._INDEX, index); - } - if (type != null) { - builder.field(Fields._TYPE, type); - } - if (id != null) { - builder.field(Fields._ID, id); - } - } - if (version != -1) { - builder.field(Fields._VERSION, version); - } - if (Float.isNaN(score)) { - builder.nullField(Fields._SCORE); - } else { - builder.field(Fields._SCORE, score); - } - for (SearchHitField field : metaFields) { - Object value = field.value(); - builder.field(field.name(), value); - } - if (source != null) { - XContentHelper.writeRawField(SourceFieldMapper.NAME, source, builder, params); - } - if (!otherFields.isEmpty()) { - builder.startObject(Fields.FIELDS); - for (SearchHitField field : otherFields) { - builder.startArray(field.name()); - for (Object value : field.getValues()) { - builder.value(value); - } - builder.endArray(); - } - builder.endObject(); - } - if (highlightFields != null && !highlightFields.isEmpty()) { - builder.startObject(Fields.HIGHLIGHT); - for (HighlightField field : highlightFields.values()) { - field.toXContent(builder, params); - } - builder.endObject(); - } - sortValues.toXContent(builder, params); - if (matchedQueries.length > 0) { - builder.startArray(Fields.MATCHED_QUERIES); - for (String matchedFilter : matchedQueries) { - builder.value(matchedFilter); - } - builder.endArray(); - } - if (explanation() != null) { - builder.field(Fields._EXPLANATION); - buildExplanation(builder, explanation()); - } - if (innerHits != null) { - builder.startObject(Fields.INNER_HITS); - for (Map.Entry entry : innerHits.entrySet()) { - builder.startObject(entry.getKey()); - entry.getValue().toXContent(builder, params); - builder.endObject(); - } - builder.endObject(); - } - return builder; - } - - public static InternalSearchHit fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token; - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); - String currentFieldName = null; - String type = null, id = null; - String index = null; - float score = DEFAULT_SCORE; - long version = -1; - SearchSortValues sortValues = SearchSortValues.EMPTY; - InternalNestedIdentity nestedIdentity = null; - Map highlightFields = new HashMap<>(); - BytesReference parsedSource = null; - List matchedQueries = new ArrayList<>(); - Map fields = new HashMap<>(); - Explanation explanation = null; - ShardId shardId = null; - String nodeId = null; - Map innerHits = null; - while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token.isValue()) { - if (Fields._TYPE.equals(currentFieldName)) { - type = parser.text(); - } else if (Fields._INDEX.equals(currentFieldName)) { - index = parser.text(); - } else if (Fields._ID.equals(currentFieldName)) { - id = parser.text(); - } else if (Fields._SCORE.equals(currentFieldName)) { - score = parser.floatValue(); - } else if (Fields._VERSION.equals(currentFieldName)) { - version = parser.longValue(); - } else if (Fields._SHARD.equals(currentFieldName)) { - shardId = ShardId.fromString(parser.text()); - } else if (Fields._NODE.equals(currentFieldName)) { - nodeId = parser.text(); - } else if (MapperService.isMetadataField(currentFieldName)) { - List values = new ArrayList<>(); - values.add(parseStoredFieldsValue(parser)); - fields.put(currentFieldName, new InternalSearchHitField(currentFieldName, values)); - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else if (token == XContentParser.Token.VALUE_NULL) { - if (Fields._SCORE.equals(currentFieldName)) { - score = Float.NaN; - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else if (token == XContentParser.Token.START_OBJECT) { - if (SourceFieldMapper.NAME.equals(currentFieldName)) { - try (XContentBuilder builder = XContentBuilder.builder(parser.contentType().xContent())) { - //the original document gets slightly modified: whitespaces or pretty printing are not preserved, - //it all depends on the current builder settings - builder.copyCurrentStructure(parser); - parsedSource = builder.bytes(); - } - } else if (Fields.HIGHLIGHT.equals(currentFieldName)) { - while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - HighlightField highlightField = HighlightField.fromXContent(parser); - highlightFields.put(highlightField.getName(), highlightField); - } - } else if (Fields.FIELDS.equals(currentFieldName)) { - while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - String fieldName = parser.currentName(); - List values = new ArrayList<>(); - ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.nextToken(), parser::getTokenLocation); - while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - values.add(parseStoredFieldsValue(parser)); - } - fields.put(fieldName, new InternalSearchHitField(fieldName, values)); - } - } else if (Fields._EXPLANATION.equals(currentFieldName)) { - explanation = parseExplanation(parser); - } else if (Fields.INNER_HITS.equals(currentFieldName)) { - innerHits = new HashMap<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - // parse the key - ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser::getTokenLocation); - String name = parser.currentName(); - innerHits.put(name, InternalSearchHits.fromXContent(parser)); - parser.nextToken(); - ensureExpectedToken(XContentParser.Token.END_OBJECT, parser.currentToken(), parser::getTokenLocation); - } - } else if (InternalNestedIdentity.Fields._NESTED.equals(currentFieldName)) { - nestedIdentity = InternalNestedIdentity.fromXContent(parser); - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else if (token == XContentParser.Token.START_ARRAY) { - if (Fields.SORT.equals(currentFieldName)) { - sortValues = SearchSortValues.fromXContent(parser); - } else if (Fields.MATCHED_QUERIES.equals(currentFieldName)) { - while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - matchedQueries.add(parser.text()); - } - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else { - throwUnknownToken(token, parser.getTokenLocation()); - } - } - InternalSearchHit internalSearchHit = new InternalSearchHit(-1, id, new Text(type), nestedIdentity, Collections.emptyMap()); - internalSearchHit.index = index; - internalSearchHit.score(score); - internalSearchHit.version(version); - internalSearchHit.sortValues(sortValues); - internalSearchHit.highlightFields(highlightFields); - internalSearchHit.sourceRef(parsedSource); - internalSearchHit.explanation(explanation); - internalSearchHit.setInnerHits(innerHits); - if (matchedQueries.size() > 0) { - internalSearchHit.matchedQueries(matchedQueries.toArray(new String[matchedQueries.size()])); - } - if (shardId != null && nodeId != null) { - internalSearchHit.shard(new SearchShardTarget(nodeId, shardId)); - } - internalSearchHit.fields(fields); - return internalSearchHit; - } - - private static Explanation parseExplanation(XContentParser parser) throws IOException { - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); - XContentParser.Token token; - Float value = null; - String description = null; - List details = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, () -> parser.getTokenLocation()); - String currentFieldName = parser.currentName(); - token = parser.nextToken(); - if (Fields.VALUE.equals(currentFieldName)) { - value = parser.floatValue(); - } else if (Fields.DESCRIPTION.equals(currentFieldName)) { - description = parser.textOrNull(); - } else if (Fields.DETAILS.equals(currentFieldName)) { - ensureExpectedToken(XContentParser.Token.START_ARRAY, token, () -> parser.getTokenLocation()); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - details.add(parseExplanation(parser)); - } - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } - if (value == null) { - throw new ParsingException(parser.getTokenLocation(), "missing explanation value"); - } - if (description == null) { - throw new ParsingException(parser.getTokenLocation(), "missing explanation description"); - } - return Explanation.match(value, description, details); - } - - private void buildExplanation(XContentBuilder builder, Explanation explanation) throws IOException { - builder.startObject(); - builder.field(Fields.VALUE, explanation.getValue()); - builder.field(Fields.DESCRIPTION, explanation.getDescription()); - Explanation[] innerExps = explanation.getDetails(); - if (innerExps != null) { - builder.startArray(Fields.DETAILS); - for (Explanation exp : innerExps) { - buildExplanation(builder, exp); - } - builder.endArray(); - } - builder.endObject(); - - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - toInnerXContent(builder, params); - builder.endObject(); - return builder; - } - - public static InternalSearchHit readSearchHit(StreamInput in) throws IOException { - InternalSearchHit hit = new InternalSearchHit(); - hit.readFrom(in); - return hit; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - score = in.readFloat(); - id = in.readOptionalText(); - type = in.readOptionalText(); - nestedIdentity = in.readOptionalWriteable(InternalNestedIdentity::new); - version = in.readLong(); - source = in.readBytesReference(); - if (source.length() == 0) { - source = null; - } - if (in.readBoolean()) { - explanation = readExplanation(in); - } - int size = in.readVInt(); - if (size == 0) { - fields = emptyMap(); - } else if (size == 1) { - SearchHitField hitField = readSearchHitField(in); - fields = singletonMap(hitField.name(), hitField); - } else { - Map fields = new HashMap<>(); - for (int i = 0; i < size; i++) { - SearchHitField hitField = readSearchHitField(in); - fields.put(hitField.name(), hitField); - } - this.fields = unmodifiableMap(fields); - } - - size = in.readVInt(); - if (size == 0) { - highlightFields = emptyMap(); - } else if (size == 1) { - HighlightField field = readHighlightField(in); - highlightFields = singletonMap(field.name(), field); - } else { - Map highlightFields = new HashMap<>(); - for (int i = 0; i < size; i++) { - HighlightField field = readHighlightField(in); - highlightFields.put(field.name(), field); - } - this.highlightFields = unmodifiableMap(highlightFields); - } - - sortValues = new SearchSortValues(in); - - size = in.readVInt(); - if (size > 0) { - matchedQueries = new String[size]; - for (int i = 0; i < size; i++) { - matchedQueries[i] = in.readString(); - } - } - // we call the setter here because that also sets the local index parameter - shard(in.readOptionalWriteable(SearchShardTarget::new)); - size = in.readVInt(); - if (size > 0) { - innerHits = new HashMap<>(size); - for (int i = 0; i < size; i++) { - String key = in.readString(); - InternalSearchHits value = InternalSearchHits.readSearchHits(in); - innerHits.put(key, value); - } - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeFloat(score); - out.writeOptionalText(id); - out.writeOptionalText(type); - out.writeOptionalWriteable(nestedIdentity); - out.writeLong(version); - out.writeBytesReference(source); - if (explanation == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - writeExplanation(out, explanation); - } - if (fields == null) { - out.writeVInt(0); - } else { - out.writeVInt(fields.size()); - for (SearchHitField hitField : fields().values()) { - hitField.writeTo(out); - } - } - if (highlightFields == null) { - out.writeVInt(0); - } else { - out.writeVInt(highlightFields.size()); - for (HighlightField highlightField : highlightFields.values()) { - highlightField.writeTo(out); - } - } - sortValues.writeTo(out); - - if (matchedQueries.length == 0) { - out.writeVInt(0); - } else { - out.writeVInt(matchedQueries.length); - for (String matchedFilter : matchedQueries) { - out.writeString(matchedFilter); - } - } - out.writeOptionalWriteable(shard); - if (innerHits == null) { - out.writeVInt(0); - } else { - out.writeVInt(innerHits.size()); - for (Map.Entry entry : innerHits.entrySet()) { - out.writeString(entry.getKey()); - entry.getValue().writeTo(out); - } - } - } - - @Override - public boolean equals(Object obj) { - if (obj == null || getClass() != obj.getClass()) { - return false; - } - InternalSearchHit other = (InternalSearchHit) obj; - return Objects.equals(id, other.id) - && Objects.equals(type, other.type) - && Objects.equals(nestedIdentity, other.nestedIdentity) - && Objects.equals(version, other.version) - && Objects.equals(source, other.source) - && Objects.equals(fields, other.fields) - && Objects.equals(highlightFields(), other.highlightFields()) - && Arrays.equals(matchedQueries, other.matchedQueries) - && Objects.equals(explanation, other.explanation) - && Objects.equals(shard, other.shard) - && Objects.equals(innerHits, other.innerHits); - } - - @Override - public int hashCode() { - return Objects.hash(id, type, nestedIdentity, version, source, fields, highlightFields(), Arrays.hashCode(matchedQueries), - explanation, shard, innerHits); - } - - public static final class InternalNestedIdentity implements NestedIdentity, Writeable, ToXContent { - - private Text field; - private int offset; - private InternalNestedIdentity child; - - public InternalNestedIdentity(String field, int offset, InternalNestedIdentity child) { - this.field = new Text(field); - this.offset = offset; - this.child = child; - } - - InternalNestedIdentity(StreamInput in) throws IOException { - field = in.readOptionalText(); - offset = in.readInt(); - child = in.readOptionalWriteable(InternalNestedIdentity::new); - } - - @Override - public Text getField() { - return field; - } - - @Override - public int getOffset() { - return offset; - } - - @Override - public NestedIdentity getChild() { - return child; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalText(field); - out.writeInt(offset); - out.writeOptionalWriteable(child); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field(Fields._NESTED); - return innerToXContent(builder, params); - } - - /** - * Rendering of the inner XContent object without the leading field name. This way the structure innerToXContent renders and - * fromXContent parses correspond to each other. - */ - XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - if (field != null) { - builder.field(Fields._NESTED_FIELD, field); - } - if (offset != -1) { - builder.field(Fields._NESTED_OFFSET, offset); - } - if (child != null) { - builder = child.toXContent(builder, params); - } - builder.endObject(); - return builder; - } - - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "nested_identity", - ctorArgs -> new InternalNestedIdentity((String) ctorArgs[0], (int) ctorArgs[1], (InternalNestedIdentity) ctorArgs[2])); - static { - PARSER.declareString(constructorArg(), new ParseField(Fields._NESTED_FIELD)); - PARSER.declareInt(constructorArg(), new ParseField(Fields._NESTED_OFFSET)); - PARSER.declareObject(optionalConstructorArg(), PARSER, new ParseField(Fields._NESTED)); - } - - public static InternalNestedIdentity fromXContent(XContentParser parser) { - return PARSER.apply(parser, null); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - InternalNestedIdentity other = (InternalNestedIdentity) obj; - return Objects.equals(field, other.field) && - Objects.equals(offset, other.offset) && - Objects.equals(child, other.child); - } - - @Override - public int hashCode() { - return Objects.hash(field, offset, child); - } - - public static class Fields { - static final String _NESTED = "_nested"; - static final String _NESTED_FIELD = "field"; - static final String _NESTED_OFFSET = "offset"; - } - } -} diff --git a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHitField.java b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHitField.java deleted file mode 100644 index d1f94969bd3..00000000000 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHitField.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.internal; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.search.SearchHitField; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - -public class InternalSearchHitField implements SearchHitField { - - private String name; - private List values; - - private InternalSearchHitField() { - } - - public InternalSearchHitField(String name, List values) { - this.name = name; - this.values = values; - } - - @Override - public String name() { - return name; - } - - @Override - public String getName() { - return name(); - } - - @Override - public Object value() { - if (values == null || values.isEmpty()) { - return null; - } - return values.get(0); - } - - @Override - public Object getValue() { - return value(); - } - - @Override - public List values() { - return values; - } - - @Override - public List getValues() { - return values(); - } - - @Override - public boolean isMetadataField() { - return MapperService.isMetadataField(name); - } - - @Override - public Iterator iterator() { - return values.iterator(); - } - - public static InternalSearchHitField readSearchHitField(StreamInput in) throws IOException { - InternalSearchHitField result = new InternalSearchHitField(); - result.readFrom(in); - return result; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - name = in.readString(); - int size = in.readVInt(); - values = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - values.add(in.readGenericValue()); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(name); - out.writeVInt(values.size()); - for (Object value : values) { - out.writeGenericValue(value); - } - } - - @Override - public boolean equals(Object obj) { - if (obj == null || getClass() != obj.getClass()) { - return false; - } - InternalSearchHitField other = (InternalSearchHitField) obj; - return Objects.equals(name, other.name) - && Objects.equals(values, other.values); - } - - @Override - public int hashCode() { - return Objects.hash(name, values); - } -} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHits.java b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHits.java deleted file mode 100644 index 834476b543a..00000000000 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchHits.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.internal; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.SearchShardTarget; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - -import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; -import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; -import static org.elasticsearch.search.internal.InternalSearchHit.readSearchHit; - -public class InternalSearchHits implements SearchHits { - - public static InternalSearchHits empty() { - // We shouldn't use static final instance, since that could directly be returned by native transport clients - return new InternalSearchHits(EMPTY, 0, 0); - } - - public static final InternalSearchHit[] EMPTY = new InternalSearchHit[0]; - - private InternalSearchHit[] hits; - - public long totalHits; - - private float maxScore; - - InternalSearchHits() { - - } - - public InternalSearchHits(InternalSearchHit[] hits, long totalHits, float maxScore) { - this.hits = hits; - this.totalHits = totalHits; - this.maxScore = maxScore; - } - - public void shardTarget(SearchShardTarget shardTarget) { - for (InternalSearchHit hit : hits) { - hit.shard(shardTarget); - } - } - - @Override - public long totalHits() { - return totalHits; - } - - @Override - public long getTotalHits() { - return totalHits(); - } - - @Override - public float maxScore() { - return this.maxScore; - } - - @Override - public float getMaxScore() { - return maxScore(); - } - - @Override - public SearchHit[] hits() { - return this.hits; - } - - @Override - public SearchHit getAt(int position) { - return hits[position]; - } - - @Override - public SearchHit[] getHits() { - return hits(); - } - - @Override - public Iterator iterator() { - return Arrays.stream(hits()).iterator(); - } - - public InternalSearchHit[] internalHits() { - return this.hits; - } - - static final class Fields { - static final String HITS = "hits"; - static final String TOTAL = "total"; - static final String MAX_SCORE = "max_score"; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(Fields.HITS); - builder.field(Fields.TOTAL, totalHits); - if (Float.isNaN(maxScore)) { - builder.nullField(Fields.MAX_SCORE); - } else { - builder.field(Fields.MAX_SCORE, maxScore); - } - builder.field(Fields.HITS); - builder.startArray(); - for (SearchHit hit : hits) { - hit.toXContent(builder, params); - } - builder.endArray(); - builder.endObject(); - return builder; - } - - public static InternalSearchHits fromXContent(XContentParser parser) throws IOException { - if (parser.currentToken() != XContentParser.Token.START_OBJECT) { - parser.nextToken(); - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser::getTokenLocation); - } - XContentParser.Token token = parser.currentToken(); - String currentFieldName = null; - List hits = new ArrayList<>(); - long totalHits = 0; - float maxScore = 0f; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token.isValue()) { - if (Fields.TOTAL.equals(currentFieldName)) { - totalHits = parser.longValue(); - } else if (Fields.MAX_SCORE.equals(currentFieldName)) { - maxScore = parser.floatValue(); - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else if (token == XContentParser.Token.VALUE_NULL) { - if (Fields.MAX_SCORE.equals(currentFieldName)) { - maxScore = Float.NaN; // NaN gets rendered as null-field - } else { - throwUnknownField(currentFieldName, parser.getTokenLocation()); - } - } else if (token == XContentParser.Token.START_ARRAY) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - hits.add(InternalSearchHit.fromXContent(parser)); - } - } - } - InternalSearchHits internalSearchHits = new InternalSearchHits(hits.toArray(new InternalSearchHit[hits.size()]), totalHits, - maxScore); - return internalSearchHits; - } - - - public static InternalSearchHits readSearchHits(StreamInput in) throws IOException { - InternalSearchHits hits = new InternalSearchHits(); - hits.readFrom(in); - return hits; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - totalHits = in.readVLong(); - maxScore = in.readFloat(); - int size = in.readVInt(); - if (size == 0) { - hits = EMPTY; - } else { - hits = new InternalSearchHit[size]; - for (int i = 0; i < hits.length; i++) { - hits[i] = readSearchHit(in); - } - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVLong(totalHits); - out.writeFloat(maxScore); - out.writeVInt(hits.length); - if (hits.length > 0) { - for (InternalSearchHit hit : hits) { - hit.writeTo(out); - } - } - } - - @Override - public boolean equals(Object obj) { - if (obj == null || getClass() != obj.getClass()) { - return false; - } - InternalSearchHits other = (InternalSearchHits) obj; - return Objects.equals(totalHits, other.totalHits) - && Objects.equals(maxScore, other.maxScore) - && Arrays.equals(hits, other.hits); - } - - @Override - public int hashCode() { - return Objects.hash(totalHits, maxScore, Arrays.hashCode(hits)); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java index 871d176ffcd..d396d14e983 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java +++ b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.internal; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -36,15 +35,13 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; -import static org.elasticsearch.search.internal.InternalSearchHits.readSearchHits; - public class InternalSearchResponse implements Streamable, ToXContent { public static InternalSearchResponse empty() { - return new InternalSearchResponse(InternalSearchHits.empty(), null, null, null, false, null); + return new InternalSearchResponse(SearchHits.empty(), null, null, null, false, null); } - private InternalSearchHits hits; + private SearchHits hits; private InternalAggregations aggregations; @@ -59,7 +56,7 @@ public class InternalSearchResponse implements Streamable, ToXContent { private InternalSearchResponse() { } - public InternalSearchResponse(InternalSearchHits hits, InternalAggregations aggregations, Suggest suggest, + public InternalSearchResponse(SearchHits hits, InternalAggregations aggregations, Suggest suggest, SearchProfileShardResults profileResults, boolean timedOut, Boolean terminatedEarly) { this.hits = hits; this.aggregations = aggregations; @@ -125,7 +122,7 @@ public class InternalSearchResponse implements Streamable, ToXContent { @Override public void readFrom(StreamInput in) throws IOException { - hits = readSearchHits(in); + hits = SearchHits.readSearchHits(in); if (in.readBoolean()) { aggregations = InternalAggregations.readAggregations(in); } diff --git a/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java b/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java index c86c0565225..1d8da832717 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java @@ -25,12 +25,10 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.suggest.Suggest; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; @@ -194,7 +192,7 @@ public final class CompletionSuggestion extends Suggest.Suggestion> contexts; private ScoreDoc doc; - private InternalSearchHit hit; + private SearchHit hit; public Option(int docID, Text text, float score, Map> contexts) { super(text, score); @@ -221,7 +219,7 @@ public final class CompletionSuggestion extends Suggest.Suggestion(contextSize); diff --git a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java index 27f9b31389c..0f3812c0cd6 100644 --- a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java @@ -550,7 +550,7 @@ public class IndicesRequestIT extends ESIntegTestCase { SearchRequest searchRequest = new SearchRequest(randomIndicesOrAliases).searchType(SearchType.QUERY_THEN_FETCH); SearchResponse searchResponse = internalCluster().coordOnlyNodeClient().search(searchRequest).actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), greaterThan(0L)); + assertThat(searchResponse.getHits().getTotalHits(), greaterThan(0L)); clearInterceptedActions(); assertSameIndices(searchRequest, SearchTransportService.QUERY_ACTION_NAME, SearchTransportService.FETCH_ID_ACTION_NAME); @@ -571,7 +571,7 @@ public class IndicesRequestIT extends ESIntegTestCase { SearchRequest searchRequest = new SearchRequest(randomIndicesOrAliases).searchType(SearchType.DFS_QUERY_THEN_FETCH); SearchResponse searchResponse = internalCluster().coordOnlyNodeClient().search(searchRequest).actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), greaterThan(0L)); + assertThat(searchResponse.getHits().getTotalHits(), greaterThan(0L)); clearInterceptedActions(); assertSameIndices(searchRequest, SearchTransportService.DFS_ACTION_NAME, SearchTransportService.QUERY_ID_ACTION_NAME, diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index 7c81746c616..48df23ea3ae 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -94,8 +94,6 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.emptyCollectionOf; -import static org.hamcrest.Matchers.endsWith; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; @@ -735,13 +733,13 @@ public class TasksIT extends ESIntegTestCase { .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.action", taskInfo.getAction()))) .get(); - assertEquals(1L, searchResponse.getHits().totalHits()); + assertEquals(1L, searchResponse.getHits().getTotalHits()); searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX).setTypes(TaskResultsService.TASK_TYPE) .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.node", taskInfo.getTaskId().getNodeId()))) .get(); - assertEquals(1L, searchResponse.getHits().totalHits()); + assertEquals(1L, searchResponse.getHits().getTotalHits()); GetTaskResponse getResponse = expectFinishedTask(taskId); assertEquals(result, getResponse.getTask().getResponseAsMap()); diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index 87249bc8b5a..8fef3d6f824 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -152,7 +152,7 @@ public class BulkProcessorRetryIT extends ESIntegTestCase { .setQuery(QueryBuilders.matchAllQuery()) .setSize(0) .get(); - assertThat(results.getHits().totalHits(), searchResultCount); + assertThat(results.getHits().getTotalHits(), searchResultCount); } private static void indexDocs(BulkProcessor processor, int numDocs) { diff --git a/core/src/test/java/org/elasticsearch/action/bulk/byscroll/AsyncBulkByScrollActionTests.java b/core/src/test/java/org/elasticsearch/action/bulk/byscroll/AsyncBulkByScrollActionTests.java index 6b3ca0d2815..92ce5040e37 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/byscroll/AsyncBulkByScrollActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/byscroll/AsyncBulkByScrollActionTests.java @@ -34,11 +34,6 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse.Failure; -import org.elasticsearch.action.bulk.byscroll.AbstractAsyncBulkByScrollAction; -import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequest; -import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse; -import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource; -import org.elasticsearch.action.bulk.byscroll.WorkingBulkByScrollTask; import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.Hit; import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.SearchFailure; import org.elasticsearch.action.bulk.BulkRequest; @@ -73,8 +68,8 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskManager; @@ -449,8 +444,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { assertThat(client.lastScroll.get().request.scroll().keepAlive().seconds(), either(equalTo(110L)).or(equalTo(109L))); // Now we can simulate a response and check the delay that we used for the task - InternalSearchHit hit = new InternalSearchHit(0, "id", new Text("type"), emptyMap()); - InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[] { hit }, 0, 0); + SearchHit hit = new SearchHit(0, "id", new Text("type"), emptyMap()); + SearchHits hits = new SearchHits(new SearchHit[] { hit }, 0, 0); InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false); SearchResponse searchResponse = new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), null); diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index db729cac1c2..1686a3c6de2 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -28,8 +28,8 @@ import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.index.Index; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; @@ -202,10 +202,10 @@ public class SearchPhaseControllerTests extends ESTestCase { float maxScore = -1F; SearchShardTarget shardTarget = new SearchShardTarget("", new Index("", ""), shardIndex); FetchSearchResult fetchSearchResult = new FetchSearchResult(shardIndex, shardTarget); - List internalSearchHits = new ArrayList<>(); + List searchHits = new ArrayList<>(); for (ScoreDoc scoreDoc : mergedSearchDocs) { if (scoreDoc.shardIndex == shardIndex) { - internalSearchHits.add(new InternalSearchHit(scoreDoc.doc, "", new Text(""), Collections.emptyMap())); + searchHits.add(new SearchHit(scoreDoc.doc, "", new Text(""), Collections.emptyMap())); if (scoreDoc.score > maxScore) { maxScore = scoreDoc.score; } @@ -216,7 +216,7 @@ public class SearchPhaseControllerTests extends ESTestCase { for (CompletionSuggestion.Entry.Option option : ((CompletionSuggestion) suggestion).getOptions()) { ScoreDoc doc = option.getDoc(); if (doc.shardIndex == shardIndex) { - internalSearchHits.add(new InternalSearchHit(doc.doc, "", new Text(""), Collections.emptyMap())); + searchHits.add(new SearchHit(doc.doc, "", new Text(""), Collections.emptyMap())); if (doc.score > maxScore) { maxScore = doc.score; } @@ -224,8 +224,8 @@ public class SearchPhaseControllerTests extends ESTestCase { } } } - InternalSearchHit[] hits = internalSearchHits.toArray(new InternalSearchHit[internalSearchHits.size()]); - fetchSearchResult.hits(new InternalSearchHits(hits, hits.length, maxScore)); + SearchHit[] hits = searchHits.toArray(new SearchHit[searchHits.size()]); + fetchSearchResult.hits(new SearchHits(hits, hits.length, maxScore)); fetchResults.set(shardIndex, fetchSearchResult); } return fetchResults; diff --git a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java index 497b134ebe3..6959758b110 100644 --- a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java +++ b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java @@ -272,32 +272,32 @@ public class IndexAliasesIT extends ESIntegTestCase { logger.info("--> checking filtering alias for two indices"); SearchResponse searchResponse = client().prepareSearch("foos").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "1", "5"); - assertThat(client().prepareSearch("foos").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("foos").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(2L)); logger.info("--> checking filtering alias for one index"); searchResponse = client().prepareSearch("bars").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "2"); - assertThat(client().prepareSearch("bars").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch("bars").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(1L)); logger.info("--> checking filtering alias for two indices and one complete index"); searchResponse = client().prepareSearch("foos", "test1").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "1", "2", "3", "4", "5"); - assertThat(client().prepareSearch("foos", "test1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(5L)); + assertThat(client().prepareSearch("foos", "test1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(5L)); logger.info("--> checking filtering alias for two indices and non-filtering alias for one index"); searchResponse = client().prepareSearch("foos", "aliasToTest1").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "1", "2", "3", "4", "5"); - assertThat(client().prepareSearch("foos", "aliasToTest1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(5L)); + assertThat(client().prepareSearch("foos", "aliasToTest1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(5L)); logger.info("--> checking filtering alias for two indices and non-filtering alias for both indices"); searchResponse = client().prepareSearch("foos", "aliasToTests").setQuery(QueryBuilders.matchAllQuery()).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(8L)); - assertThat(client().prepareSearch("foos", "aliasToTests").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(8L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(8L)); + assertThat(client().prepareSearch("foos", "aliasToTests").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(8L)); logger.info("--> checking filtering alias for two indices and non-filtering alias for both indices"); searchResponse = client().prepareSearch("foos", "aliasToTests").setQuery(QueryBuilders.termQuery("name", "something")).get(); assertHits(searchResponse.getHits(), "4", "8"); - assertThat(client().prepareSearch("foos", "aliasToTests").setSize(0).setQuery(QueryBuilders.termQuery("name", "something")).get().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("foos", "aliasToTests").setSize(0).setQuery(QueryBuilders.termQuery("name", "something")).get().getHits().getTotalHits(), equalTo(2L)); } public void testSearchingFilteringAliasesMultipleIndices() throws Exception { @@ -341,27 +341,27 @@ public class IndexAliasesIT extends ESIntegTestCase { logger.info("--> checking filtering alias for multiple indices"); SearchResponse searchResponse = client().prepareSearch("filter23", "filter13").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "21", "31", "13", "33"); - assertThat(client().prepareSearch("filter23", "filter13").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(4L)); + assertThat(client().prepareSearch("filter23", "filter13").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(4L)); searchResponse = client().prepareSearch("filter23", "filter1").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "21", "31", "11", "12", "13"); - assertThat(client().prepareSearch("filter23", "filter1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(5L)); + assertThat(client().prepareSearch("filter23", "filter1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(5L)); searchResponse = client().prepareSearch("filter13", "filter1").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "11", "12", "13", "33"); - assertThat(client().prepareSearch("filter13", "filter1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(4L)); + assertThat(client().prepareSearch("filter13", "filter1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(4L)); searchResponse = client().prepareSearch("filter13", "filter1", "filter23").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "11", "12", "13", "21", "31", "33"); - assertThat(client().prepareSearch("filter13", "filter1", "filter23").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(6L)); + assertThat(client().prepareSearch("filter13", "filter1", "filter23").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(6L)); searchResponse = client().prepareSearch("filter23", "filter13", "test2").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "21", "22", "23", "31", "13", "33"); - assertThat(client().prepareSearch("filter23", "filter13", "test2").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(6L)); + assertThat(client().prepareSearch("filter23", "filter13", "test2").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(6L)); searchResponse = client().prepareSearch("filter23", "filter13", "test1", "test2").setQuery(QueryBuilders.matchAllQuery()).get(); assertHits(searchResponse.getHits(), "11", "12", "13", "21", "22", "23", "31", "33"); - assertThat(client().prepareSearch("filter23", "filter13", "test1", "test2").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(8L)); + assertThat(client().prepareSearch("filter23", "filter13", "test1", "test2").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(8L)); } public void testDeletingByQueryFilteringAliases() throws Exception { @@ -398,7 +398,7 @@ public class IndexAliasesIT extends ESIntegTestCase { refresh(); logger.info("--> checking counts before delete"); - assertThat(client().prepareSearch("bars").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch("bars").setSize(0).setQuery(QueryBuilders.matchAllQuery()).get().getHits().getTotalHits(), equalTo(1L)); } public void testDeleteAliases() throws Exception { @@ -838,10 +838,10 @@ public class IndexAliasesIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch("filter1").get(); assertHitCount(response, 1); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); response = client().prepareSearch("filter2").get(); assertHitCount(response, 1); - assertThat(response.getHits().getAt(0).id(), equalTo("2")); + assertThat(response.getHits().getAt(0).getId(), equalTo("2")); } public void testAliasesWithBlocks() { @@ -920,10 +920,10 @@ public class IndexAliasesIT extends ESIntegTestCase { } private void assertHits(SearchHits hits, String... ids) { - assertThat(hits.totalHits(), equalTo((long) ids.length)); + assertThat(hits.getTotalHits(), equalTo((long) ids.length)); Set hitIds = new HashSet<>(); for (SearchHit hit : hits.getHits()) { - hitIds.add(hit.id()); + hitIds.add(hit.getId()); } assertThat(hitIds, containsInAnyOrder(ids)); } diff --git a/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java b/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java index 63091a97818..9eb42075eda 100644 --- a/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java +++ b/core/src/test/java/org/elasticsearch/broadcast/BroadcastActionsIT.java @@ -56,7 +56,7 @@ public class BroadcastActionsIT extends ESIntegTestCase { SearchResponse countResponse = client().prepareSearch("test").setSize(0) .setQuery(termQuery("_type", "type1")) .get(); - assertThat(countResponse.getHits().totalHits(), equalTo(2L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(2L)); assertThat(countResponse.getTotalShards(), equalTo(numShards.numPrimaries)); assertThat(countResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries)); assertThat(countResponse.getFailedShards(), equalTo(0)); diff --git a/core/src/test/java/org/elasticsearch/bwcompat/IpFieldBwCompatIT.java b/core/src/test/java/org/elasticsearch/bwcompat/IpFieldBwCompatIT.java index 5f830d2edd8..e1aa8d1425d 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/IpFieldBwCompatIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/IpFieldBwCompatIT.java @@ -61,7 +61,7 @@ public class IpFieldBwCompatIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch("old_index", "new_index") .addSort(SortBuilders.fieldSort("ip_field")).get(); assertNoFailures(response); - assertEquals(3, response.getHits().totalHits()); + assertEquals(3, response.getHits().getTotalHits()); assertEquals("::1", response.getHits().getAt(0).getSortValues()[0]); assertEquals("127.0.0.1", response.getHits().getAt(1).getSortValues()[0]); assertEquals("127.0.0.1", response.getHits().getAt(2).getSortValues()[0]); @@ -73,7 +73,7 @@ public class IpFieldBwCompatIT extends ESIntegTestCase { .addMaskRange("127.0.0.1/16") .addMaskRange("::1/64")).get(); assertNoFailures(response); - assertEquals(3, response.getHits().totalHits()); + assertEquals(3, response.getHits().getTotalHits()); Range range = response.getAggregations().get("ip_range"); assertEquals(2, range.getBuckets().size()); assertEquals("::1/64", range.getBuckets().get(0).getKeyAsString()); @@ -86,7 +86,7 @@ public class IpFieldBwCompatIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch("old_index", "new_index") .addAggregation(AggregationBuilders.terms("ip_terms").field("ip_field")).get(); assertNoFailures(response); - assertEquals(3, response.getHits().totalHits()); + assertEquals(3, response.getHits().getTotalHits()); Terms terms = response.getAggregations().get("ip_terms"); assertEquals(2, terms.getBuckets().size()); assertEquals(2, terms.getBucketByKey("127.0.0.1").getDocCount()); diff --git a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java index 76e9a4eb105..553ab15d670 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java @@ -346,7 +346,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase { // Make sure there are payloads and they are taken into account for the score // the 'string' field has a boost of 4 in the mappings so it should get a payload boost - String stringValue = (String) bestHit.sourceAsMap().get("string"); + String stringValue = (String) bestHit.getSourceAsMap().get("string"); assertNotNull(stringValue); Explanation explanation = client().prepareExplain(indexName, bestHit.getType(), bestHit.getId()) .setQuery(QueryBuilders.matchQuery("_all", stringValue)).get().getExplanation(); @@ -457,7 +457,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase { } // We can read from the alias just like we can read from the index. String aliasName = "#" + indexName; - long totalDocs = client().prepareSearch(indexName).setSize(0).get().getHits().totalHits(); + long totalDocs = client().prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(); assertHitCount(client().prepareSearch(aliasName).setSize(0).get(), totalDocs); assertThat(totalDocs, greaterThanOrEqualTo(2000L)); @@ -475,11 +475,11 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase { builder.setSize(100); builder.addStoredField("binary"); SearchHits hits = builder.get().getHits(); - assertEquals(100, hits.hits().length); + assertEquals(100, hits.getHits().length); for(SearchHit hit : hits) { SearchHitField field = hit.field("binary"); assertNotNull(field); - Object value = field.value(); + Object value = field.getValue(); assertTrue(value instanceof BytesArray); assertEquals(16, ((BytesArray) value).length()); } diff --git a/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java index 02ba7f5ea4f..92c8b2315cc 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java @@ -86,7 +86,7 @@ public class RepositoryUpgradabilityIT extends AbstractSnapshotIntegTestCase { assertThat(restoredIndices, equalTo(indices)); // make sure it has documents for (final String searchIdx : restoredIndices) { - assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(), greaterThan(0L)); + assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().getTotalHits(), greaterThan(0L)); } deleteIndices(restoredIndices); // delete so we can restore again later @@ -134,7 +134,7 @@ public class RepositoryUpgradabilityIT extends AbstractSnapshotIntegTestCase { Set oldRestoredIndices = Sets.newHashSet(restoreSnapshot(repoName, originalSnapshot.snapshotId().getName())); assertThat(oldRestoredIndices, equalTo(Sets.newHashSet(originalIndex))); for (final String searchIdx : oldRestoredIndices) { - assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(), + assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().getTotalHits(), greaterThanOrEqualTo((long)addedDocSize)); } deleteIndices(oldRestoredIndices); @@ -143,7 +143,7 @@ public class RepositoryUpgradabilityIT extends AbstractSnapshotIntegTestCase { Set newSnapshotIndices = Sets.newHashSet(restoreSnapshot(repoName, snapshotName3)); assertThat(newSnapshotIndices, equalTo(Sets.newHashSet(originalIndex, indexName2))); for (final String searchIdx : newSnapshotIndices) { - assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(), + assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().getTotalHits(), greaterThanOrEqualTo((long)addedDocSize)); } deleteIndices(newSnapshotIndices); // clean up indices before starting again diff --git a/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java index 9dd9e9dd4a6..228991f787b 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java @@ -172,7 +172,7 @@ public class RestoreBackwardsCompatIT extends AbstractSnapshotIntegTestCase { logger.info("--> check search"); SearchResponse searchResponse = client().prepareSearch(index).get(); - assertThat(searchResponse.getHits().totalHits(), greaterThan(1L)); + assertThat(searchResponse.getHits().getTotalHits(), greaterThan(1L)); logger.info("--> check settings"); ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); diff --git a/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java b/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java index d63203eda25..450744d64df 100644 --- a/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java @@ -25,7 +25,6 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.ElectMasterService; @@ -37,7 +36,6 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.discovery.TestZenDiscovery; import org.elasticsearch.test.disruption.NetworkDisruption; -import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDelay; import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; @@ -123,7 +121,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase { logger.info("--> verify we the data back"); for (int i = 0; i < 10; i++) { - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(100L)); } internalCluster().stopCurrentMasterNode(); diff --git a/core/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java b/core/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java index 93771019ef4..49c63362f58 100644 --- a/core/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java @@ -60,7 +60,7 @@ public class FilteringAllocationIT extends ESIntegTestCase { client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet(); } client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(100L)); logger.info("--> decommission the second node"); client().admin().cluster().prepareUpdateSettings() @@ -79,7 +79,7 @@ public class FilteringAllocationIT extends ESIntegTestCase { } client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(100L)); } public void testDisablingAllocationFiltering() throws Exception { @@ -101,7 +101,7 @@ public class FilteringAllocationIT extends ESIntegTestCase { client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet(); } client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(100L)); ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); IndexRoutingTable indexRoutingTable = clusterState.routingTable().index("test"); int numShardsOnNode1 = 0; diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 5e2869a658e..784ba65796d 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -1142,7 +1142,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { // wait for relocation to finish endRelocationLatch.await(); // now search for the documents and see if we get a reply - assertThat(client().prepareSearch().setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch().setSize(0).get().getHits().getTotalHits(), equalTo(100L)); } public void testIndexImportedFromDataOnlyNodesIfMasterLostDataFolder() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java b/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java index 34b4bc05727..37370292d47 100644 --- a/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java +++ b/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java @@ -162,14 +162,14 @@ public class DocumentActionsIT extends ESIntegTestCase { // test successful SearchResponse countResponse = client().prepareSearch("test").setSize(0).setQuery(termQuery("_type", "type1")).execute().actionGet(); assertNoFailures(countResponse); - assertThat(countResponse.getHits().totalHits(), equalTo(2L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(2L)); assertThat(countResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries)); assertThat(countResponse.getFailedShards(), equalTo(0)); // count with no query is a match all one countResponse = client().prepareSearch("test").setSize(0).execute().actionGet(); assertThat("Failures " + countResponse.getShardFailures(), countResponse.getShardFailures() == null ? 0 : countResponse.getShardFailures().length, equalTo(0)); - assertThat(countResponse.getHits().totalHits(), equalTo(2L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(2L)); assertThat(countResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries)); assertThat(countResponse.getFailedShards(), equalTo(0)); } diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java index e713f14be14..2bf9f0efbfd 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java @@ -684,7 +684,7 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase { SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).addDocValueField("foo").addSort("foo", SortOrder.ASC).get(); assertHitCount(resp, 4); assertOrderedSearchHits(resp, "2", "3", "4", "1"); - SearchHit[] hits = resp.getHits().hits(); + SearchHit[] hits = resp.getHits().getHits(); assertThat(hits[0].field("foo").getValue().toString(), equalTo("bar")); assertThat(hits[1].field("foo").getValue().toString(), equalTo("baz")); assertThat(hits[2].field("foo").getValue().toString(), equalTo("eggplant")); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/CopyToMapperIntegrationIT.java b/core/src/test/java/org/elasticsearch/index/mapper/CopyToMapperIntegrationIT.java index 3cf9527c627..c78bd336528 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/CopyToMapperIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/CopyToMapperIntegrationIT.java @@ -60,7 +60,7 @@ public class CopyToMapperIntegrationIT extends ESIntegTestCase { .collectMode(aggCollectionMode)) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) recordCount)); + assertThat(response.getHits().getTotalHits(), equalTo((long) recordCount)); assertThat(((Terms) response.getAggregations().get("test")).getBuckets().size(), equalTo(recordCount + 1)); assertThat(((Terms) response.getAggregations().get("test_raw")).getBuckets().size(), equalTo(recordCount)); @@ -84,7 +84,7 @@ public class CopyToMapperIntegrationIT extends ESIntegTestCase { client().admin().indices().prepareRefresh("test-idx").execute().actionGet(); SearchResponse response = client().prepareSearch("test-idx") .setQuery(QueryBuilders.termQuery("root.top.child", "bar")).get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); } private XContentBuilder createDynamicTemplateMapping() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ExternalValuesMapperIntegrationIT.java b/core/src/test/java/org/elasticsearch/index/mapper/ExternalValuesMapperIntegrationIT.java index bc2d6c564ec..f0109e10abb 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ExternalValuesMapperIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ExternalValuesMapperIntegrationIT.java @@ -109,25 +109,25 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase { .setPostFilter(QueryBuilders.termQuery("field.bool", "true")) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) 1)); + assertThat(response.getHits().getTotalHits(), equalTo((long) 1)); response = client().prepareSearch("test-idx") .setPostFilter(QueryBuilders.geoDistanceQuery("field.point").point(42.0, 51.0).distance("1km")) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) 1)); + assertThat(response.getHits().getTotalHits(), equalTo((long) 1)); response = client().prepareSearch("test-idx") .setPostFilter(QueryBuilders.geoShapeQuery("field.shape", ShapeBuilders.newPoint(-100, 45)).relation(ShapeRelation.WITHIN)) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) 1)); + assertThat(response.getHits().getTotalHits(), equalTo((long) 1)); response = client().prepareSearch("test-idx") .setPostFilter(QueryBuilders.termQuery("field.field", "foo")) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) 1)); + assertThat(response.getHits().getTotalHits(), equalTo((long) 1)); } public void testExternalValuesWithMultifield() throws Exception { @@ -157,6 +157,6 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase { .setQuery(QueryBuilders.termQuery("f.g.raw", "FOO BAR")) .execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) 1)); + assertThat(response.getHits().getTotalHits(), equalTo((long) 1)); } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java index ec620814797..5b3ec34e885 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java @@ -18,20 +18,16 @@ */ package org.elasticsearch.index.mapper; -import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Priority; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.geo.GeoPoint; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.InternalSettingsPlugin; -import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.geo.RandomGeoGenerator; import org.hamcrest.CoreMatchers; @@ -245,11 +241,11 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { // TODO these tests are bogus and need to be Fix // query by geohash subfield SearchResponse searchResponse = client().prepareSearch().addStoredField("location.geohash").setQuery(matchAllQuery()).execute().actionGet(); - assertEquals(numDocs, searchResponse.getHits().totalHits()); + assertEquals(numDocs, searchResponse.getHits().getTotalHits()); // query by latlon subfield searchResponse = client().prepareSearch().addStoredField("location.latlon").setQuery(matchAllQuery()).execute().actionGet(); - assertEquals(numDocs, searchResponse.getHits().totalHits()); + assertEquals(numDocs, searchResponse.getHits().getTotalHits()); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldsIntegrationIT.java b/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldsIntegrationIT.java index 1df01218a50..ae922e6a731 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldsIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldsIntegrationIT.java @@ -66,11 +66,11 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("my-index") .setQuery(matchQuery("title", "multi")) .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("my-index") .setQuery(matchQuery("title.not_analyzed", "Multi fields")) .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); assertAcked( client().admin().indices().preparePutMapping("my-index").setType("my-type") @@ -97,7 +97,7 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { searchResponse = client().prepareSearch("my-index") .setQuery(matchQuery("title.uncased", "Multi")) .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testGeoPointMultiField() throws Exception { @@ -125,9 +125,9 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { SearchResponse countResponse = client().prepareSearch("my-index").setSize(0) .setQuery(constantScoreQuery(geoDistanceQuery("a").point(51, 19).distance(50, DistanceUnit.KILOMETERS))) .get(); - assertThat(countResponse.getHits().totalHits(), equalTo(1L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(1L)); countResponse = client().prepareSearch("my-index").setSize(0).setQuery(matchQuery("a.b", point.geohash())).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(1L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(1L)); } public void testTokenCountMultiField() throws Exception { @@ -163,7 +163,7 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { client().prepareIndex("my-index", "my-type", "1").setSource("a", "my tokens").setRefreshPolicy(IMMEDIATE).get(); SearchResponse countResponse = client().prepareSearch("my-index").setSize(0).setQuery(matchQuery("a.b", "my tokens")).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(1L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(1L)); } public void testCompletionMultiField() throws Exception { @@ -187,7 +187,7 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { client().prepareIndex("my-index", "my-type", "1").setSource("a", "complete me").setRefreshPolicy(IMMEDIATE).get(); SearchResponse countResponse = client().prepareSearch("my-index").setSize(0).setQuery(matchQuery("a.b", "complete me")).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(1L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(1L)); } public void testIpMultiField() throws Exception { @@ -211,7 +211,7 @@ public class MultiFieldsIntegrationIT extends ESIntegTestCase { client().prepareIndex("my-index", "my-type", "1").setSource("a", "127.0.0.1").setRefreshPolicy(IMMEDIATE).get(); SearchResponse countResponse = client().prepareSearch("my-index").setSize(0).setQuery(matchQuery("a.b", "127.0.0.1")).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(1L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(1L)); } private XContentBuilder createMappingSource(String fieldType) throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java b/core/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java index a84a18d2aef..48307b6349c 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/TokenCountFieldMapperIntegrationIT.java @@ -177,14 +177,14 @@ public class TokenCountFieldMapperIntegrationIT extends ESIntegTestCase { private void assertSearchReturns(SearchResponse result, String... ids) { assertThat(result.getHits().getTotalHits(), equalTo((long) ids.length)); - assertThat(result.getHits().hits().length, equalTo(ids.length)); + assertThat(result.getHits().getHits().length, equalTo(ids.length)); List foundIds = new ArrayList<>(); for (SearchHit hit : result.getHits()) { - foundIds.add(hit.id()); + foundIds.add(hit.getId()); } assertThat(foundIds, containsInAnyOrder(ids)); for (SearchHit hit : result.getHits()) { - String id = hit.id(); + String id = hit.getId(); if (id.equals("single")) { assertSearchHit(hit, 4); } else if (id.equals("bulk1")) { @@ -205,13 +205,13 @@ public class TokenCountFieldMapperIntegrationIT extends ESIntegTestCase { private void assertSearchHit(SearchHit hit, int... termCounts) { assertThat(hit.field("foo.token_count"), not(nullValue())); - assertThat(hit.field("foo.token_count").values().size(), equalTo(termCounts.length)); + assertThat(hit.field("foo.token_count").getValues().size(), equalTo(termCounts.length)); for (int i = 0; i < termCounts.length; i++) { - assertThat((Integer) hit.field("foo.token_count").values().get(i), equalTo(termCounts[i])); + assertThat((Integer) hit.field("foo.token_count").getValues().get(i), equalTo(termCounts[i])); } if (loadCountedFields && storeCountedFields) { - assertThat(hit.field("foo").values().size(), equalTo(termCounts.length)); + assertThat(hit.field("foo").getValues().size(), equalTo(termCounts.length)); } } } diff --git a/core/src/test/java/org/elasticsearch/index/query/CommonTermsQueryParserTests.java b/core/src/test/java/org/elasticsearch/index/query/CommonTermsQueryParserTests.java index 6b217f68eea..f4e737ea4b0 100644 --- a/core/src/test/java/org/elasticsearch/index/query/CommonTermsQueryParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/CommonTermsQueryParserTests.java @@ -45,6 +45,6 @@ public class CommonTermsQueryParserTests extends ESSingleNodeTestCase { client().prepareSearch(index).setTypes(type).setQuery(commonTermsQueryBuilder).execute().actionGet(); assertNotNull(response); - assertEquals(response.getHits().hits().length, 0); + assertEquals(response.getHits().getHits().length, 0); } } diff --git a/core/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java b/core/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java index 373937784b3..24825b8a99a 100644 --- a/core/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java +++ b/core/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java @@ -30,7 +30,6 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.engine.SegmentsStats; @@ -39,8 +38,6 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; @@ -139,7 +136,7 @@ public class ExceptionRetryIT extends ESIntegTestCase { if (!found_duplicate_already) { SearchResponse dupIdResponse = client().prepareSearch("index").setQuery(termQuery("_id", searchResponse.getHits().getHits()[i].getId())).setExplain(true).get(); - assertThat(dupIdResponse.getHits().totalHits(), greaterThan(1L)); + assertThat(dupIdResponse.getHits().getTotalHits(), greaterThan(1L)); logger.info("found a duplicate id:"); for (SearchHit hit : dupIdResponse.getHits()) { logger.info("Doc {} was found on shard {}", hit.getId(), hit.getShard().getShardId()); diff --git a/core/src/test/java/org/elasticsearch/indexing/IndexActionIT.java b/core/src/test/java/org/elasticsearch/indexing/IndexActionIT.java index 43326116f15..9b5cb7a138d 100644 --- a/core/src/test/java/org/elasticsearch/indexing/IndexActionIT.java +++ b/core/src/test/java/org/elasticsearch/indexing/IndexActionIT.java @@ -18,28 +18,20 @@ */ package org.elasticsearch.indexing; -import org.elasticsearch.Version; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.indices.InvalidIndexNameException; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.InternalSettingsPlugin; -import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.test.junit.annotations.TestLogging; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Random; @@ -78,8 +70,8 @@ public class IndexActionIT extends ESIntegTestCase { try { logger.debug("running search with all types"); SearchResponse response = client().prepareSearch("test").get(); - if (response.getHits().totalHits() != numOfDocs) { - final String message = "Count is " + response.getHits().totalHits() + " but " + numOfDocs + " was expected. " + if (response.getHits().getTotalHits() != numOfDocs) { + final String message = "Count is " + response.getHits().getTotalHits() + " but " + numOfDocs + " was expected. " + ElasticsearchAssertions.formatShardStatus(response); logger.error("{}. search response: \n{}", message, response); fail(message); @@ -93,8 +85,8 @@ public class IndexActionIT extends ESIntegTestCase { try { logger.debug("running search with a specific type"); SearchResponse response = client().prepareSearch("test").setTypes("type").get(); - if (response.getHits().totalHits() != numOfDocs) { - final String message = "Count is " + response.getHits().totalHits() + " but " + numOfDocs + " was expected. " + if (response.getHits().getTotalHits() != numOfDocs) { + final String message = "Count is " + response.getHits().getTotalHits() + " but " + numOfDocs + " was expected. " + ElasticsearchAssertions.formatShardStatus(response); logger.error("{}. search response: \n{}", message, response); fail(message); diff --git a/core/src/test/java/org/elasticsearch/indices/flush/FlushIT.java b/core/src/test/java/org/elasticsearch/indices/flush/FlushIT.java index 50979cf611d..0416dbb8855 100644 --- a/core/src/test/java/org/elasticsearch/indices/flush/FlushIT.java +++ b/core/src/test/java/org/elasticsearch/indices/flush/FlushIT.java @@ -184,12 +184,12 @@ public class FlushIT extends ESIntegTestCase { indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); assertFlushResponseEqualsShardStats(indexStats.getShards(), syncedFlushResult.getShardsResultPerIndex().get("test")); refresh(); - assertThat(client().prepareSearch().setSize(0).get().getHits().totalHits(), equalTo((long) numDocs.get())); - logger.info("indexed {} docs", client().prepareSearch().setSize(0).get().getHits().totalHits()); + assertThat(client().prepareSearch().setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs.get())); + logger.info("indexed {} docs", client().prepareSearch().setSize(0).get().getHits().getTotalHits()); logClusterState(); internalCluster().fullRestart(); ensureGreen(); - assertThat(client().prepareSearch().setSize(0).get().getHits().totalHits(), equalTo((long) numDocs.get())); + assertThat(client().prepareSearch().setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs.get())); } private void assertFlushResponseEqualsShardStats(ShardStats[] shardsStats, List syncedFlushResults) { diff --git a/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationIT.java b/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationIT.java index ee05e0ae526..39262fd19b7 100644 --- a/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationIT.java @@ -85,7 +85,7 @@ public class UpdateMappingIntegrationIT extends ESIntegTestCase { RefreshResponse refreshResponse = client().admin().indices().prepareRefresh().execute().actionGet(); assertThat(refreshResponse.getFailedShards(), equalTo(0)); SearchResponse response = client().prepareSearch("test").setSize(0).execute().actionGet(); - assertThat(response.getHits().totalHits(), equalTo((long) recCount)); + assertThat(response.getHits().getTotalHits(), equalTo((long) recCount)); logger.info("checking all the fields are in the mappings"); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 5a7f983b056..48c8c2e5f28 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -537,7 +537,7 @@ public class IndexRecoveryIT extends ESIntegTestCase { indexRandom(true, docs); flush(); - assertThat(client().prepareSearch(name).setSize(0).get().getHits().totalHits(), equalTo((long) numDocs)); + assertThat(client().prepareSearch(name).setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs)); return client().admin().indices().prepareStats(name).execute().actionGet(); } diff --git a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java index cb6b8b752bc..eb6a83b9a91 100644 --- a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java @@ -128,7 +128,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 1); - assertThat(searchResponse.getHits().getAt(0).field("field1").value().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getAt(0).field("field1").getValue().toString(), equalTo("value1")); // field2 is not stored. assertThat(searchResponse.getHits().getAt(0).field("field2"), nullValue()); @@ -146,8 +146,8 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { logger.warn("failed search {}", Arrays.toString(searchResponse.getShardFailures())); } assertHitCount(searchResponse, 1); - assertThat(searchResponse.getHits().getAt(0).field("field1").value().toString(), equalTo("value1")); - assertThat(searchResponse.getHits().getAt(0).field("field2").value().toString(), equalTo("value 2")); + assertThat(searchResponse.getHits().getAt(0).field("field1").getValue().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getAt(0).field("field2").getValue().toString(), equalTo("value 2")); } public void testDeleteIndexTemplate() throws Exception { @@ -409,7 +409,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { searchResponse = client().prepareSearch("filtered_alias").get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type2")); + assertThat(searchResponse.getHits().getAt(0).getType(), equalTo("type2")); // Search the complex filter alias searchResponse = client().prepareSearch("complex_filtered_alias").get(); @@ -455,7 +455,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { searchResponse = client().prepareSearch("my_alias").get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type2")); + assertThat(searchResponse.getHits().getAt(0).getType(), equalTo("type2")); } public void testIndexTemplateWithAliasesSource() { @@ -493,7 +493,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { searchResponse = client().prepareSearch("alias2").get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type2")); + assertThat(searchResponse.getHits().getAt(0).getType(), equalTo("type2")); } public void testDuplicateAlias() throws Exception { @@ -769,7 +769,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 1); - assertEquals("value1", searchResponse.getHits().getAt(0).field("field1").value().toString()); + assertEquals("value1", searchResponse.getHits().getAt(0).field("field1").getValue().toString()); assertNull(searchResponse.getHits().getAt(0).field("field2")); // bx -> matches template @@ -780,7 +780,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 1); - assertEquals("value1", searchResponse.getHits().getAt(0).field("field1").value().toString()); + assertEquals("value1", searchResponse.getHits().getAt(0).field("field1").getValue().toString()); assertNull(searchResponse.getHits().getAt(0).field("field2")); } diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java index d0dd969d0c7..57249e186db 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java @@ -276,7 +276,7 @@ public class RecoveryWhileUnderLoadIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch().setSize((int) numberOfDocs).setQuery(matchAllQuery()).addSort("id", SortOrder.ASC).get(); logSearchResponse(numberOfShards, numberOfDocs, i, searchResponse); iterationResults[i] = searchResponse; - if (searchResponse.getHits().totalHits() != numberOfDocs) { + if (searchResponse.getHits().getTotalHits() != numberOfDocs) { error = true; } } @@ -312,7 +312,7 @@ public class RecoveryWhileUnderLoadIT extends ESIntegTestCase { boolean errorOccurred = false; for (int i = 0; i < iterations; i++) { SearchResponse searchResponse = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(); - if (searchResponse.getHits().totalHits() != numberOfDocs) { + if (searchResponse.getHits().getTotalHits() != numberOfDocs) { errorOccurred = true; } } @@ -337,7 +337,7 @@ public class RecoveryWhileUnderLoadIT extends ESIntegTestCase { if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) { logger.info("iteration [{}] - shard failures: {}", iteration, Arrays.toString(searchResponse.getShardFailures())); } - logger.info("iteration [{}] - returned documents: {} (expected {})", iteration, searchResponse.getHits().totalHits(), numberOfDocs); + logger.info("iteration [{}] - returned documents: {} (expected {})", iteration, searchResponse.getHits().getTotalHits(), numberOfDocs); } private void refreshAndAssert() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index f32688aafff..8848a61c976 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -165,7 +165,7 @@ public class RelocationIT extends ESIntegTestCase { logger.info("--> verifying count"); client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().totalHits(), equalTo(20L)); + assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().getTotalHits(), equalTo(20L)); logger.info("--> start another node"); final String node_2 = internalCluster().startNode(); @@ -182,7 +182,7 @@ public class RelocationIT extends ESIntegTestCase { logger.info("--> verifying count again..."); client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().totalHits(), equalTo(20L)); + assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().getTotalHits(), equalTo(20L)); } @TestLogging("org.elasticsearch.action.bulk:TRACE,org.elasticsearch.action.search:TRACE") @@ -256,14 +256,14 @@ public class RelocationIT extends ESIntegTestCase { logger.info("--> START search test round {}", i + 1); SearchHits hits = client().prepareSearch("test").setQuery(matchAllQuery()).setSize((int) indexer.totalIndexedDocs()).storedFields().execute().actionGet().getHits(); ranOnce = true; - if (hits.totalHits() != indexer.totalIndexedDocs()) { + if (hits.getTotalHits() != indexer.totalIndexedDocs()) { int[] hitIds = new int[(int) indexer.totalIndexedDocs()]; for (int hit = 0; hit < indexer.totalIndexedDocs(); hit++) { hitIds[hit] = hit + 1; } IntHashSet set = IntHashSet.from(hitIds); - for (SearchHit hit : hits.hits()) { - int id = Integer.parseInt(hit.id()); + for (SearchHit hit : hits.getHits()) { + int id = Integer.parseInt(hit.getId()); if (!set.remove(id)) { logger.error("Extra id [{}]", id); } @@ -272,7 +272,7 @@ public class RelocationIT extends ESIntegTestCase { logger.error("Missing id [{}]", value); }); } - assertThat(hits.totalHits(), equalTo(indexer.totalIndexedDocs())); + assertThat(hits.getTotalHits(), equalTo(indexer.totalIndexedDocs())); logger.info("--> DONE search test round {}", i + 1); } @@ -369,9 +369,9 @@ public class RelocationIT extends ESIntegTestCase { SearchResponse response = client.prepareSearch("test").setPreference("_local").setSize(0).get(); assertNoFailures(response); if (expectedCount < 0) { - expectedCount = response.getHits().totalHits(); + expectedCount = response.getHits().getTotalHits(); } else { - assertEquals(expectedCount, response.getHits().totalHits()); + assertEquals(expectedCount, response.getHits().getTotalHits()); } } diff --git a/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java index dc8b6b54464..b99353812c8 100644 --- a/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java @@ -124,23 +124,23 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> search with no routing, should fine one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with wrong routing, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); } logger.info("--> search with correct routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> indexing with id [2], and routing [1] using alias"); @@ -148,50 +148,50 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with 0 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with 1 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with 0,1 routings , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with two routing aliases , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with alias0, alias1 and alias01, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias0", "alias1", "alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias0", "alias1", "alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1", "alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1", "alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with test, alias0 and alias1, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("test", "alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("test", "alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("test", "alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("test", "alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } } @@ -233,20 +233,20 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> search with alias-a1,alias-b0, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-a1", "alias-b0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch("alias-a1", "alias-b0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias-a1", "alias-b0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias-a1", "alias-b0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); } logger.info("--> search with alias-ab, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-ab").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias-ab").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias-ab").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias-ab").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with alias-a0,alias-b1 should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-a0", "alias-b1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias-a0", "alias-b1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias-a0", "alias-b1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias-a0", "alias-b1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } } @@ -270,7 +270,7 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> search all on index_* should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("index_*").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("index_*").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } } @@ -295,7 +295,7 @@ public class AliasRoutingIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("index_*").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(1).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(); logger.info("--> search all on index_* should find two"); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); //Let's make sure that, even though 2 docs are available, only one is returned according to the size we set in the request //Therefore the reduce phase has taken place, which proves that the QUERY_AND_FETCH search type wasn't erroneously forced. assertThat(searchResponse.getHits().getHits().length, equalTo(1)); @@ -315,8 +315,8 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> verifying get and search with routing, should find"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "0").setRouting("3").execute().actionGet().isExists(), equalTo(true)); - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> creating alias with routing [4]"); @@ -325,8 +325,8 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> verifying search with wrong routing should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); } logger.info("--> creating alias with search routing [3,4] and index routing 4"); @@ -341,8 +341,8 @@ public class AliasRoutingIT extends ESIntegTestCase { for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "0").setRouting("3").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet("test", "type1", "1").setRouting("4").execute().actionGet().isExists(), equalTo(true)); - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } } diff --git a/core/src/test/java/org/elasticsearch/routing/PartitionedRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/PartitionedRoutingIT.java index 14634116f04..9ceedb2378b 100644 --- a/core/src/test/java/org/elasticsearch/routing/PartitionedRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/PartitionedRoutingIT.java @@ -124,11 +124,11 @@ public class PartitionedRoutingIT extends ESIntegTestCase { .execute().actionGet(); logger.info("--> routed search on index [" + index + "] visited [" + response.getTotalShards() - + "] shards for routing [" + routing + "] and got hits [" + response.getHits().totalHits() + "]"); + + "] shards for routing [" + routing + "] and got hits [" + response.getHits().getTotalHits() + "]"); assertTrue(response.getTotalShards() + " was not in " + expectedShards + " for " + index, expectedShards.contains(response.getTotalShards())); - assertEquals(expectedDocuments, response.getHits().totalHits()); + assertEquals(expectedDocuments, response.getHits().getTotalHits()); Set found = new HashSet<>(); response.getHits().forEach(h -> found.add(h.getId())); @@ -149,7 +149,7 @@ public class PartitionedRoutingIT extends ESIntegTestCase { .execute().actionGet(); assertEquals(expectedShards, response.getTotalShards()); - assertEquals(expectedDocuments, response.getHits().totalHits()); + assertEquals(expectedDocuments, response.getHits().getTotalHits()); Set found = new HashSet<>(); response.getHits().forEach(h -> found.add(h.getId())); diff --git a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index b027a13e867..031bac145a3 100644 --- a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -114,19 +114,19 @@ public class SimpleRoutingIT extends ESIntegTestCase { logger.info("--> search with no routing, should fine one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with wrong routing, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); } logger.info("--> search with correct routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> indexing with id [2], and routing [1]"); @@ -134,32 +134,32 @@ public class SimpleRoutingIT extends ESIntegTestCase { logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with 0 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with 1 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); + assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(1L)); } logger.info("--> search with 0,1 routings , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } logger.info("--> search with 0,1,0 routings , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0", "1", "0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0", "1", "0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setRouting("0", "1", "0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); + assertThat(client().prepareSearch().setSize(0).setRouting("0", "1", "0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(2L)); } } diff --git a/core/src/test/java/org/elasticsearch/script/IndexLookupIT.java b/core/src/test/java/org/elasticsearch/script/IndexLookupIT.java index f41d9d7c394..981aab15ea4 100644 --- a/core/src/test/java/org/elasticsearch/script/IndexLookupIT.java +++ b/core/src/test/java/org/elasticsearch/script/IndexLookupIT.java @@ -606,7 +606,7 @@ public class IndexLookupIT extends ESIntegTestCase { assertHitCount(sr, numExpectedDocs); for (SearchHit hit : sr.getHits().getHits()) { assertThat("for doc " + hit.getId(), ((Float) expectedScore.get(hit.getId())).doubleValue(), - Matchers.closeTo(hit.score(), 1.e-4)); + Matchers.closeTo(hit.getScore(), 1.e-4)); } } @@ -967,7 +967,7 @@ public class IndexLookupIT extends ESIntegTestCase { try { SearchResponse sr = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).addScriptField("tvtest", script) .execute().actionGet(); - assertThat(sr.getHits().hits().length, equalTo(0)); + assertThat(sr.getHits().getHits().length, equalTo(0)); ShardSearchFailure[] shardFails = sr.getShardFailures(); for (ShardSearchFailure fail : shardFails) { assertThat(fail.reason().indexOf("Cannot iterate twice! If you want to iterate more that once, add _CACHE explicitly."), @@ -992,7 +992,7 @@ public class IndexLookupIT extends ESIntegTestCase { Object expectedResult = expectedFieldVals.get(hit.getId()); assertThat("for doc " + hit.getId(), result, equalTo(expectedResult)); assertThat("for doc " + hit.getId(), ((Float) expectedScore.get(hit.getId())).doubleValue(), - Matchers.closeTo(hit.score(), 1.e-4)); + Matchers.closeTo(hit.getScore(), 1.e-4)); } } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptFieldIT.java b/core/src/test/java/org/elasticsearch/script/ScriptFieldIT.java index 1a4a58e4b1e..28f39fb3f13 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptFieldIT.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptFieldIT.java @@ -65,7 +65,7 @@ public class ScriptFieldIT extends ESIntegTestCase { .addScriptField("float", new Script(ScriptType.INLINE, "native", "float", Collections.emptyMap())) .addScriptField("double", new Script(ScriptType.INLINE, "native", "double", Collections.emptyMap())) .addScriptField("long", new Script(ScriptType.INLINE, "native", "long", Collections.emptyMap())).execute().actionGet(); - assertThat(sr.getHits().hits().length, equalTo(6)); + assertThat(sr.getHits().getHits().length, equalTo(6)); for (SearchHit hit : sr.getHits().getHits()) { Object result = hit.getFields().get("int").getValues().get(0); assertThat(result, equalTo((Object) intArray)); diff --git a/core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitTests.java b/core/src/test/java/org/elasticsearch/search/InternalSearchHitTests.java similarity index 71% rename from core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitTests.java rename to core/src/test/java/org/elasticsearch/search/InternalSearchHitTests.java index ea2fb1d0db3..80746f6873a 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitTests.java +++ b/core/src/test/java/org/elasticsearch/search/InternalSearchHitTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.internal; +package org.elasticsearch.search; import org.apache.lucene.search.Explanation; import org.elasticsearch.common.bytes.BytesArray; @@ -34,11 +34,9 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.SearchHitField; -import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; import org.elasticsearch.search.fetch.subphase.highlight.HighlightFieldTests; -import org.elasticsearch.search.internal.InternalSearchHit.InternalNestedIdentity; +import org.elasticsearch.search.SearchHit.NestedIdentity; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.RandomObjects; @@ -61,13 +59,13 @@ public class InternalSearchHitTests extends ESTestCase { private static Set META_FIELDS = Sets.newHashSet("_uid", "_all", "_parent", "_routing", "_size", "_timestamp", "_ttl"); - public static InternalSearchHit createTestItem(boolean withOptionalInnerHits) { + public static SearchHit createTestItem(boolean withOptionalInnerHits) { int internalId = randomInt(); String uid = randomAsciiOfLength(10); Text type = new Text(randomAsciiOfLengthBetween(5, 10)); - InternalNestedIdentity nestedIdentity = null; + NestedIdentity nestedIdentity = null; if (randomBoolean()) { - nestedIdentity = InternalNestedIdentityTests.createTestItem(randomIntBetween(0, 2)); + nestedIdentity = NestedIdentityTests.createTestItem(randomIntBetween(0, 2)); } Map fields = new HashMap<>(); if (randomBoolean()) { @@ -77,14 +75,14 @@ public class InternalSearchHitTests extends ESTestCase { XContentType.JSON); if (randomBoolean()) { String metaField = randomFrom(META_FIELDS); - fields.put(metaField, new InternalSearchHitField(metaField, values.v1())); + fields.put(metaField, new SearchHitField(metaField, values.v1())); } else { String fieldName = randomAsciiOfLengthBetween(5, 10); - fields.put(fieldName, new InternalSearchHitField(fieldName, values.v1())); + fields.put(fieldName, new SearchHitField(fieldName, values.v1())); } } } - InternalSearchHit hit = new InternalSearchHit(internalId, uid, type, nestedIdentity, fields); + SearchHit hit = new SearchHit(internalId, uid, type, nestedIdentity, fields); if (frequently()) { if (rarely()) { hit.score(Float.NaN); @@ -122,7 +120,7 @@ public class InternalSearchHitTests extends ESTestCase { } if (withOptionalInnerHits) { int innerHitsSize = randomIntBetween(0, 3); - Map innerHits = new HashMap<>(innerHitsSize); + Map innerHits = new HashMap<>(innerHitsSize); for (int i = 0; i < innerHitsSize; i++) { innerHits.put(randomAsciiOfLength(5), InternalSearchHitsTests.createTestItem()); } @@ -136,15 +134,15 @@ public class InternalSearchHitTests extends ESTestCase { } public void testFromXContent() throws IOException { - InternalSearchHit searchHit = createTestItem(true); + SearchHit searchHit = createTestItem(true); boolean humanReadable = randomBoolean(); XContentType xContentType = randomFrom(XContentType.values()); BytesReference originalBytes = toXContent(searchHit, xContentType, humanReadable); - InternalSearchHit parsed; + SearchHit parsed; try (XContentParser parser = createParser(xContentType.xContent(), originalBytes)) { parser.nextToken(); // jump to first START_OBJECT - parsed = InternalSearchHit.fromXContent(parser); + parsed = SearchHit.fromXContent(parser); assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken()); assertNull(parser.nextToken()); } @@ -152,66 +150,66 @@ public class InternalSearchHitTests extends ESTestCase { } public void testToXContent() throws IOException { - InternalSearchHit internalSearchHit = new InternalSearchHit(1, "id1", new Text("type"), Collections.emptyMap()); - internalSearchHit.score(1.5f); + SearchHit searchHit = new SearchHit(1, "id1", new Text("type"), Collections.emptyMap()); + searchHit.score(1.5f); XContentBuilder builder = JsonXContent.contentBuilder(); - internalSearchHit.toXContent(builder, ToXContent.EMPTY_PARAMS); + searchHit.toXContent(builder, ToXContent.EMPTY_PARAMS); assertEquals("{\"_type\":\"type\",\"_id\":\"id1\",\"_score\":1.5}", builder.string()); } public void testSerializeShardTarget() throws Exception { SearchShardTarget target = new SearchShardTarget("_node_id", new Index("_index", "_na_"), 0); - Map innerHits = new HashMap<>(); - InternalSearchHit innerHit1 = new InternalSearchHit(0, "_id", new Text("_type"), null); + Map innerHits = new HashMap<>(); + SearchHit innerHit1 = new SearchHit(0, "_id", new Text("_type"), null); innerHit1.shard(target); - InternalSearchHit innerInnerHit2 = new InternalSearchHit(0, "_id", new Text("_type"), null); + SearchHit innerInnerHit2 = new SearchHit(0, "_id", new Text("_type"), null); innerInnerHit2.shard(target); - innerHits.put("1", new InternalSearchHits(new InternalSearchHit[]{innerInnerHit2}, 1, 1f)); + innerHits.put("1", new SearchHits(new SearchHit[]{innerInnerHit2}, 1, 1f)); innerHit1.setInnerHits(innerHits); - InternalSearchHit innerHit2 = new InternalSearchHit(0, "_id", new Text("_type"), null); + SearchHit innerHit2 = new SearchHit(0, "_id", new Text("_type"), null); innerHit2.shard(target); - InternalSearchHit innerHit3 = new InternalSearchHit(0, "_id", new Text("_type"), null); + SearchHit innerHit3 = new SearchHit(0, "_id", new Text("_type"), null); innerHit3.shard(target); innerHits = new HashMap<>(); - InternalSearchHit hit1 = new InternalSearchHit(0, "_id", new Text("_type"), null); - innerHits.put("1", new InternalSearchHits(new InternalSearchHit[]{innerHit1, innerHit2}, 1, 1f)); - innerHits.put("2", new InternalSearchHits(new InternalSearchHit[]{innerHit3}, 1, 1f)); + SearchHit hit1 = new SearchHit(0, "_id", new Text("_type"), null); + innerHits.put("1", new SearchHits(new SearchHit[]{innerHit1, innerHit2}, 1, 1f)); + innerHits.put("2", new SearchHits(new SearchHit[]{innerHit3}, 1, 1f)); hit1.shard(target); hit1.setInnerHits(innerHits); - InternalSearchHit hit2 = new InternalSearchHit(0, "_id", new Text("_type"), null); + SearchHit hit2 = new SearchHit(0, "_id", new Text("_type"), null); hit2.shard(target); - InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[]{hit1, hit2}, 2, 1f); + SearchHits hits = new SearchHits(new SearchHit[]{hit1, hit2}, 2, 1f); BytesStreamOutput output = new BytesStreamOutput(); hits.writeTo(output); InputStream input = output.bytes().streamInput(); - InternalSearchHits results = InternalSearchHits.readSearchHits(new InputStreamStreamInput(input)); - assertThat(results.getAt(0).shard(), equalTo(target)); - assertThat(results.getAt(0).getInnerHits().get("1").getAt(0).shard(), notNullValue()); - assertThat(results.getAt(0).getInnerHits().get("1").getAt(0).getInnerHits().get("1").getAt(0).shard(), notNullValue()); - assertThat(results.getAt(0).getInnerHits().get("1").getAt(1).shard(), notNullValue()); - assertThat(results.getAt(0).getInnerHits().get("2").getAt(0).shard(), notNullValue()); - assertThat(results.getAt(1).shard(), equalTo(target)); + SearchHits results = SearchHits.readSearchHits(new InputStreamStreamInput(input)); + assertThat(results.getAt(0).getShard(), equalTo(target)); + assertThat(results.getAt(0).getInnerHits().get("1").getAt(0).getShard(), notNullValue()); + assertThat(results.getAt(0).getInnerHits().get("1").getAt(0).getInnerHits().get("1").getAt(0).getShard(), notNullValue()); + assertThat(results.getAt(0).getInnerHits().get("1").getAt(1).getShard(), notNullValue()); + assertThat(results.getAt(0).getInnerHits().get("2").getAt(0).getShard(), notNullValue()); + assertThat(results.getAt(1).getShard(), equalTo(target)); } public void testNullSource() throws Exception { - InternalSearchHit searchHit = new InternalSearchHit(0, "_id", new Text("_type"), null); + SearchHit searchHit = new SearchHit(0, "_id", new Text("_type"), null); - assertThat(searchHit.source(), nullValue()); - assertThat(searchHit.sourceRef(), nullValue()); - assertThat(searchHit.sourceAsMap(), nullValue()); - assertThat(searchHit.sourceAsString(), nullValue()); - assertThat(searchHit.getSource(), nullValue()); + assertThat(searchHit.getSourceAsMap(), nullValue()); + assertThat(searchHit.getSourceRef(), nullValue()); + assertThat(searchHit.getSourceAsMap(), nullValue()); + assertThat(searchHit.getSourceAsString(), nullValue()); + assertThat(searchHit.getSourceAsMap(), nullValue()); assertThat(searchHit.getSourceRef(), nullValue()); assertThat(searchHit.getSourceAsString(), nullValue()); } public void testHasSource() { - InternalSearchHit searchHit = new InternalSearchHit(randomInt()); + SearchHit searchHit = new SearchHit(randomInt()); assertFalse(searchHit.hasSource()); searchHit.sourceRef(new BytesArray("{}")); assertTrue(searchHit.hasSource()); diff --git a/core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitsTests.java b/core/src/test/java/org/elasticsearch/search/InternalSearchHitsTests.java similarity index 81% rename from core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitsTests.java rename to core/src/test/java/org/elasticsearch/search/InternalSearchHitsTests.java index 2474893edf5..71632875694 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/InternalSearchHitsTests.java +++ b/core/src/test/java/org/elasticsearch/search/InternalSearchHitsTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.internal; +package org.elasticsearch.search; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.text.Text; @@ -36,25 +36,25 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXC public class InternalSearchHitsTests extends ESTestCase { - public static InternalSearchHits createTestItem() { + public static SearchHits createTestItem() { int searchHits = randomIntBetween(0, 5); - InternalSearchHit[] hits = new InternalSearchHit[searchHits]; + SearchHit[] hits = new SearchHit[searchHits]; for (int i = 0; i < searchHits; i++) { hits[i] = InternalSearchHitTests.createTestItem(false); // creating random innerHits could create loops } long totalHits = randomLong(); float maxScore = frequently() ? randomFloat() : Float.NaN; - return new InternalSearchHits(hits, totalHits, maxScore); + return new SearchHits(hits, totalHits, maxScore); } public void testFromXContent() throws IOException { - InternalSearchHits searchHits = createTestItem(); + SearchHits searchHits = createTestItem(); XContentType xcontentType = randomFrom(XContentType.values()); boolean humanReadable = randomBoolean(); BytesReference originalBytes = toXContent(searchHits, xcontentType, humanReadable); - InternalSearchHits parsed; + SearchHits parsed; try (XContentParser parser = createParser(xcontentType.xContent(), originalBytes)) { - parsed = InternalSearchHits.fromXContent(parser); + parsed = SearchHits.fromXContent(parser); assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken()); assertEquals(XContentParser.Token.END_OBJECT, parser.nextToken()); assertNull(parser.nextToken()); @@ -63,13 +63,13 @@ public class InternalSearchHitsTests extends ESTestCase { } public void testToXContent() throws IOException { - InternalSearchHit[] hits = new InternalSearchHit[] { - new InternalSearchHit(1, "id1", new Text("type"), Collections.emptyMap()), - new InternalSearchHit(2, "id2", new Text("type"), Collections.emptyMap()) }; + SearchHit[] hits = new SearchHit[] { + new SearchHit(1, "id1", new Text("type"), Collections.emptyMap()), + new SearchHit(2, "id2", new Text("type"), Collections.emptyMap()) }; long totalHits = 1000; float maxScore = 1.5f; - InternalSearchHits searchHits = new InternalSearchHits(hits, totalHits, maxScore); + SearchHits searchHits = new SearchHits(hits, totalHits, maxScore); XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); searchHits.toXContent(builder, ToXContent.EMPTY_PARAMS); diff --git a/core/src/test/java/org/elasticsearch/search/internal/InternalNestedIdentityTests.java b/core/src/test/java/org/elasticsearch/search/NestedIdentityTests.java similarity index 69% rename from core/src/test/java/org/elasticsearch/search/internal/InternalNestedIdentityTests.java rename to core/src/test/java/org/elasticsearch/search/NestedIdentityTests.java index 0621b574432..b062ace0f23 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/InternalNestedIdentityTests.java +++ b/core/src/test/java/org/elasticsearch/search/NestedIdentityTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.internal; +package org.elasticsearch.search; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; @@ -27,7 +27,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.search.internal.InternalSearchHit.InternalNestedIdentity; +import org.elasticsearch.search.SearchHit.NestedIdentity; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -37,20 +37,20 @@ import java.util.function.Supplier; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; -public class InternalNestedIdentityTests extends ESTestCase { +public class NestedIdentityTests extends ESTestCase { - public static InternalNestedIdentity createTestItem(int depth) { + public static NestedIdentity createTestItem(int depth) { String field = frequently() ? randomAsciiOfLengthBetween(1, 20) : randomRealisticUnicodeOfCodepointLengthBetween(1, 20); int offset = randomInt(10); - InternalNestedIdentity child = null; + NestedIdentity child = null; if (depth > 0) { child = createTestItem(depth - 1); } - return new InternalNestedIdentity(field, offset, child); + return new NestedIdentity(field, offset, child); } public void testFromXContent() throws IOException { - InternalNestedIdentity nestedIdentity = createTestItem(randomInt(3)); + NestedIdentity nestedIdentity = createTestItem(randomInt(3)); XContentType xcontentType = randomFrom(XContentType.values()); XContentBuilder builder = XContentFactory.contentBuilder(xcontentType); if (randomBoolean()) { @@ -58,13 +58,13 @@ public class InternalNestedIdentityTests extends ESTestCase { } builder = nestedIdentity.innerToXContent(builder, ToXContent.EMPTY_PARAMS); XContentParser parser = createParser(builder); - InternalNestedIdentity parsedNestedIdentity = InternalNestedIdentity.fromXContent(parser); + NestedIdentity parsedNestedIdentity = NestedIdentity.fromXContent(parser); assertEquals(nestedIdentity, parsedNestedIdentity); assertNull(parser.nextToken()); } public void testToXContent() throws IOException { - InternalNestedIdentity nestedIdentity = new InternalNestedIdentity("foo", 5, null); + NestedIdentity nestedIdentity = new NestedIdentity("foo", 5, null); XContentBuilder builder = JsonXContent.contentBuilder(); builder.prettyPrint(); builder.startObject(); @@ -78,7 +78,7 @@ public class InternalNestedIdentityTests extends ESTestCase { " }\n" + "}", builder.string()); - nestedIdentity = new InternalNestedIdentity("foo", 5, new InternalNestedIdentity("bar", 3, null)); + nestedIdentity = new NestedIdentity("foo", 5, new NestedIdentity("bar", 3, null)); builder = JsonXContent.contentBuilder(); builder.prettyPrint(); builder.startObject(); @@ -101,15 +101,15 @@ public class InternalNestedIdentityTests extends ESTestCase { * Test equality and hashCode properties */ public void testEqualsAndHashcode() { - checkEqualsAndHashCode(createTestItem(randomInt(3)), InternalNestedIdentityTests::copy, InternalNestedIdentityTests::mutate); + checkEqualsAndHashCode(createTestItem(randomInt(3)), NestedIdentityTests::copy, NestedIdentityTests::mutate); } public void testSerialization() throws IOException { - InternalNestedIdentity nestedIdentity = createTestItem(randomInt(3)); + NestedIdentity nestedIdentity = createTestItem(randomInt(3)); try (BytesStreamOutput output = new BytesStreamOutput()) { nestedIdentity.writeTo(output); try (StreamInput in = output.bytes().streamInput()) { - InternalNestedIdentity deserializedCopy = new InternalNestedIdentity(in); + NestedIdentity deserializedCopy = new NestedIdentity(in); assertEquals(nestedIdentity, deserializedCopy); assertEquals(nestedIdentity.hashCode(), deserializedCopy.hashCode()); assertNotSame(nestedIdentity, deserializedCopy); @@ -117,24 +117,24 @@ public class InternalNestedIdentityTests extends ESTestCase { } } - private static InternalNestedIdentity mutate(InternalNestedIdentity original) { + private static NestedIdentity mutate(NestedIdentity original) { if (original == null) { return createTestItem(0); } - List> mutations = new ArrayList<>(); + List> mutations = new ArrayList<>(); int offset = original.getOffset(); - InternalNestedIdentity child = (InternalNestedIdentity) original.getChild(); + NestedIdentity child = (NestedIdentity) original.getChild(); String fieldName = original.getField().string(); mutations.add(() -> - new InternalNestedIdentity(original.getField().string() + "_prefix", offset, child)); - mutations.add(() -> new InternalNestedIdentity(fieldName, offset + 1, child)); - mutations.add(() -> new InternalNestedIdentity(fieldName, offset, mutate(child))); + new NestedIdentity(original.getField().string() + "_prefix", offset, child)); + mutations.add(() -> new NestedIdentity(fieldName, offset + 1, child)); + mutations.add(() -> new NestedIdentity(fieldName, offset, mutate(child))); return randomFrom(mutations).get(); } - private static InternalNestedIdentity copy(InternalNestedIdentity original) { - InternalNestedIdentity child = (InternalNestedIdentity) original.getChild(); - return new InternalNestedIdentity(original.getField().string(), original.getOffset(), child != null ? copy(child) : null); + private static NestedIdentity copy(NestedIdentity original) { + NestedIdentity child = original.getChild(); + return new NestedIdentity(original.getField().string(), original.getOffset(), child != null ? copy(child) : null); } } diff --git a/core/src/test/java/org/elasticsearch/search/internal/SearchSortValuesTests.java b/core/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java similarity index 99% rename from core/src/test/java/org/elasticsearch/search/internal/SearchSortValuesTests.java rename to core/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java index 3cff30cab1c..96733a2ac6d 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/SearchSortValuesTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.internal; +package org.elasticsearch.search; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java index c8eb475c968..b16aacd9a16 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java @@ -61,12 +61,12 @@ public class AggregationsIntegrationIT extends ESIntegTestCase { assertEquals(Math.min(numDocs, 3L), terms.getBucketByKey("0").getDocCount()); int total = response.getHits().getHits().length; - while (response.getHits().hits().length > 0) { + while (response.getHits().getHits().length > 0) { response = client().prepareSearchScroll(response.getScrollId()) .setScroll(new TimeValue(500)) .execute().actionGet(); assertNull(response.getAggregations()); - total += response.getHits().hits().length; + total += response.getHits().getHits().length; } clearScroll(response.getScrollId()); assertEquals(numDocs, total); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java index 22688261004..f42341b4a02 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java @@ -190,7 +190,7 @@ public class ChildrenIT extends ESIntegTestCase { TopHits topHits = childrenBucket.getAggregations().get("top_comments"); logger.info("total_hits={}", topHits.getHits().getTotalHits()); for (SearchHit searchHit : topHits.getHits()) { - logger.info("hit= {} {} {}", searchHit.sortValues()[0], searchHit.getType(), searchHit.getId()); + logger.info("hit= {} {} {}", searchHit.getSortValues()[0], searchHit.getType(), searchHit.getId()); } } @@ -202,7 +202,7 @@ public class ChildrenIT extends ESIntegTestCase { assertThat(childrenBucket.getName(), equalTo("to_comment")); assertThat(childrenBucket.getDocCount(), equalTo(2L)); TopHits topHits = childrenBucket.getAggregations().get("top_comments"); - assertThat(topHits.getHits().totalHits(), equalTo(2L)); + assertThat(topHits.getHits().getTotalHits(), equalTo(2L)); assertThat(topHits.getHits().getAt(0).getId(), equalTo("a")); assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment")); assertThat(topHits.getHits().getAt(1).getId(), equalTo("c")); @@ -216,7 +216,7 @@ public class ChildrenIT extends ESIntegTestCase { assertThat(childrenBucket.getName(), equalTo("to_comment")); assertThat(childrenBucket.getDocCount(), equalTo(1L)); topHits = childrenBucket.getAggregations().get("top_comments"); - assertThat(topHits.getHits().totalHits(), equalTo(1L)); + assertThat(topHits.getHits().getTotalHits(), equalTo(1L)); assertThat(topHits.getHits().getAt(0).getId(), equalTo("c")); assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment")); @@ -228,7 +228,7 @@ public class ChildrenIT extends ESIntegTestCase { assertThat(childrenBucket.getName(), equalTo("to_comment")); assertThat(childrenBucket.getDocCount(), equalTo(1L)); topHits = childrenBucket.getAggregations().get("top_comments"); - assertThat(topHits.getHits().totalHits(), equalTo(1L)); + assertThat(topHits.getHits().getTotalHits(), equalTo(1L)); assertThat(topHits.getHits().getAt(0).getId(), equalTo("c")); assertThat(topHits.getHits().getAt(0).getType(), equalTo("comment")); } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java index 90342df179e..78a7fb5c176 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java @@ -1028,7 +1028,7 @@ public class DateHistogramIT extends ESIntegTestCase { ).execute().actionGet(); assertSearchResponse(response); - assertThat("Expected 24 buckets for one day aggregation with hourly interval", response.getHits().totalHits(), equalTo(2L)); + assertThat("Expected 24 buckets for one day aggregation with hourly interval", response.getHits().getTotalHits(), equalTo(2L)); Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java index de8416daf0a..ec91db0c8fc 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java @@ -181,7 +181,7 @@ public abstract class AbstractGeoTestCase extends ESIntegTestCase { SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME).addStoredField(NUMBER_FIELD_NAME).addSort(SortBuilders.fieldSort(NUMBER_FIELD_NAME) .order(SortOrder.ASC)).setSize(5000).get(); assertSearchResponse(response); - long totalHits = response.getHits().totalHits(); + long totalHits = response.getHits().getTotalHits(); XContentBuilder builder = XContentFactory.jsonBuilder(); response.toXContent(builder, ToXContent.EMPTY_PARAMS); logger.info("Full high_card_idx Response Content:\n{ {} }", builder.string()); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java index 4bc640dc900..b2adee43a4b 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java @@ -294,14 +294,14 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(3)); higestSortValue += 10; - assertThat((Long) hits.getAt(0).sortValues()[0], equalTo(higestSortValue)); - assertThat((Long) hits.getAt(1).sortValues()[0], equalTo(higestSortValue - 1)); - assertThat((Long) hits.getAt(2).sortValues()[0], equalTo(higestSortValue - 2)); + assertThat((Long) hits.getAt(0).getSortValues()[0], equalTo(higestSortValue)); + assertThat((Long) hits.getAt(1).getSortValues()[0], equalTo(higestSortValue - 1)); + assertThat((Long) hits.getAt(2).getSortValues()[0], equalTo(higestSortValue - 2)); - assertThat(hits.getAt(0).sourceAsMap().size(), equalTo(4)); + assertThat(hits.getAt(0).getSourceAsMap().size(), equalTo(4)); } } @@ -318,8 +318,8 @@ public class TopHitsIT extends ESIntegTestCase { assertSearchResponse(response); assertThat(response.getHits().getTotalHits(), equalTo(8L)); - assertThat(response.getHits().hits().length, equalTo(0)); - assertThat(response.getHits().maxScore(), equalTo(0f)); + assertThat(response.getHits().getHits().length, equalTo(0)); + assertThat(response.getHits().getMaxScore(), equalTo(0f)); Terms terms = response.getAggregations().get("terms"); assertThat(terms, notNullValue()); assertThat(terms.getName(), equalTo("terms")); @@ -354,8 +354,8 @@ public class TopHitsIT extends ESIntegTestCase { assertSearchResponse(response); assertThat(response.getHits().getTotalHits(), equalTo(8L)); - assertThat(response.getHits().hits().length, equalTo(0)); - assertThat(response.getHits().maxScore(), equalTo(0f)); + assertThat(response.getHits().getHits().length, equalTo(0)); + assertThat(response.getHits().getMaxScore(), equalTo(0f)); terms = response.getAggregations().get("terms"); assertThat(terms, notNullValue()); assertThat(terms.getName(), equalTo("terms")); @@ -386,10 +386,10 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(3)); - assertThat(hits.getAt(0).sourceAsMap().size(), equalTo(4)); + assertThat(hits.getAt(0).getSourceAsMap().size(), equalTo(4)); } } @@ -417,10 +417,10 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(3)); - assertThat(hits.getAt(0).sourceAsMap().size(), equalTo(4)); + assertThat(hits.getAt(0).getSourceAsMap().size(), equalTo(4)); id --; } } @@ -480,12 +480,12 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(controlHits.totalHits())); + assertThat(hits.getTotalHits(), equalTo(controlHits.getTotalHits())); assertThat(hits.getHits().length, equalTo(controlHits.getHits().length)); for (int i = 0; i < hits.getHits().length; i++) { - logger.info("{}: top_hits: [{}][{}] control: [{}][{}]", i, hits.getAt(i).id(), hits.getAt(i).sortValues()[0], controlHits.getAt(i).id(), controlHits.getAt(i).sortValues()[0]); - assertThat(hits.getAt(i).id(), equalTo(controlHits.getAt(i).id())); - assertThat(hits.getAt(i).sortValues()[0], equalTo(controlHits.getAt(i).sortValues()[0])); + logger.info("{}: top_hits: [{}][{}] control: [{}][{}]", i, hits.getAt(i).getId(), hits.getAt(i).getSortValues()[0], controlHits.getAt(i).getId(), controlHits.getAt(i).getSortValues()[0]); + assertThat(hits.getAt(i).getId(), equalTo(controlHits.getAt(i).getId())); + assertThat(hits.getAt(i).getSortValues()[0], equalTo(controlHits.getAt(i).getSortValues()[0])); } } @@ -517,11 +517,11 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(3)); - assertThat((Long) hits.getAt(0).sortValues()[0], equalTo(higestSortValue)); - assertThat((Long) hits.getAt(1).sortValues()[0], equalTo(higestSortValue - 1)); - assertThat((Long) hits.getAt(2).sortValues()[0], equalTo(higestSortValue - 2)); + assertThat((Long) hits.getAt(0).getSortValues()[0], equalTo(higestSortValue)); + assertThat((Long) hits.getAt(1).getSortValues()[0], equalTo(higestSortValue - 1)); + assertThat((Long) hits.getAt(2).getSortValues()[0], equalTo(higestSortValue - 2)); Max max = bucket.getAggregations().get("max_sort"); assertThat(max.getValue(), equalTo(((Long) higestSortValue).doubleValue())); higestSortValue -= 10; @@ -550,25 +550,25 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(key(bucket), equalTo("b")); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(4L)); + assertThat(hits.getTotalHits(), equalTo(4L)); assertThat(hits.getHits().length, equalTo(1)); - assertThat(hits.getAt(0).id(), equalTo("6")); + assertThat(hits.getAt(0).getId(), equalTo("6")); bucket = bucketIterator.next(); assertThat(key(bucket), equalTo("c")); topHits = bucket.getAggregations().get("hits"); hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(3L)); + assertThat(hits.getTotalHits(), equalTo(3L)); assertThat(hits.getHits().length, equalTo(1)); - assertThat(hits.getAt(0).id(), equalTo("9")); + assertThat(hits.getAt(0).getId(), equalTo("9")); bucket = bucketIterator.next(); assertThat(key(bucket), equalTo("a")); topHits = bucket.getAggregations().get("hits"); hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(2L)); + assertThat(hits.getTotalHits(), equalTo(2L)); assertThat(hits.getHits().length, equalTo(1)); - assertThat(hits.getAt(0).id(), equalTo("2")); + assertThat(hits.getAt(0).getId(), equalTo("2")); } public void testFetchFeatures() { @@ -599,7 +599,7 @@ public class TopHitsIT extends ESIntegTestCase { for (Terms.Bucket bucket : terms.getBuckets()) { TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(1)); SearchHit hit = hits.getAt(0); @@ -607,24 +607,24 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(highlightField.getFragments().length, equalTo(1)); assertThat(highlightField.getFragments()[0].string(), equalTo("some text to entertain")); - Explanation explanation = hit.explanation(); + Explanation explanation = hit.getExplanation(); assertThat(explanation.toString(), containsString("text:text")); - long version = hit.version(); + long version = hit.getVersion(); assertThat(version, equalTo(1L)); - assertThat(hit.matchedQueries()[0], equalTo("test")); + assertThat(hit.getMatchedQueries()[0], equalTo("test")); SearchHitField field = hit.field("field1"); assertThat(field.getValue().toString(), equalTo("5")); - assertThat(hit.getSource().get("text").toString(), equalTo("some text to entertain")); + assertThat(hit.getSourceAsMap().get("text").toString(), equalTo("some text to entertain")); field = hit.field("script"); assertThat(field.getValue().toString(), equalTo("5")); - assertThat(hit.sourceAsMap().size(), equalTo(1)); - assertThat(hit.sourceAsMap().get("text").toString(), equalTo("some text to entertain")); + assertThat(hit.getSourceAsMap().size(), equalTo(1)); + assertThat(hit.getSourceAsMap().get("text").toString(), equalTo("some text to entertain")); } } @@ -653,7 +653,7 @@ public class TopHitsIT extends ESIntegTestCase { TopHits hits = response.getAggregations().get("hits"); assertThat(hits, notNullValue()); assertThat(hits.getName(), equalTo("hits")); - assertThat(hits.getHits().totalHits(), equalTo(0L)); + assertThat(hits.getHits().getTotalHits(), equalTo(0L)); } public void testTrackScores() throws Exception { @@ -684,21 +684,21 @@ public class TopHitsIT extends ESIntegTestCase { TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); assertThat(hits.getMaxScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); - assertThat(hits.getAt(0).score(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); + assertThat(hits.getAt(0).getScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); bucket = terms.getBucketByKey("b"); assertThat(key(bucket), equalTo("b")); topHits = bucket.getAggregations().get("hits"); hits = topHits.getHits(); assertThat(hits.getMaxScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); - assertThat(hits.getAt(0).score(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); + assertThat(hits.getAt(0).getScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); bucket = terms.getBucketByKey("c"); assertThat(key(bucket), equalTo("c")); topHits = bucket.getAggregations().get("hits"); hits = topHits.getHits(); assertThat(hits.getMaxScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); - assertThat(hits.getAt(0).score(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); + assertThat(hits.getAt(0).getScore(), trackScore ? not(equalTo(Float.NaN)) : equalTo(Float.NaN)); } } @@ -725,31 +725,31 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(1L)); TopHits topHits = bucket.getAggregations().get("top-comments"); SearchHits searchHits = topHits.getHits(); - assertThat(searchHits.totalHits(), equalTo(1L)); + assertThat(searchHits.getTotalHits(), equalTo(1L)); assertThat(searchHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(searchHits.getAt(0).getNestedIdentity().getOffset(), equalTo(0)); - assertThat((Integer) searchHits.getAt(0).getSource().get("date"), equalTo(1)); + assertThat((Integer) searchHits.getAt(0).getSourceAsMap().get("date"), equalTo(1)); bucket = terms.getBucketByKey("b"); assertThat(bucket.getDocCount(), equalTo(2L)); topHits = bucket.getAggregations().get("top-comments"); searchHits = topHits.getHits(); - assertThat(searchHits.totalHits(), equalTo(2L)); + assertThat(searchHits.getTotalHits(), equalTo(2L)); assertThat(searchHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(searchHits.getAt(0).getNestedIdentity().getOffset(), equalTo(1)); - assertThat((Integer) searchHits.getAt(0).getSource().get("date"), equalTo(2)); + assertThat((Integer) searchHits.getAt(0).getSourceAsMap().get("date"), equalTo(2)); assertThat(searchHits.getAt(1).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(searchHits.getAt(1).getNestedIdentity().getOffset(), equalTo(0)); - assertThat((Integer) searchHits.getAt(1).getSource().get("date"), equalTo(3)); + assertThat((Integer) searchHits.getAt(1).getSourceAsMap().get("date"), equalTo(3)); bucket = terms.getBucketByKey("c"); assertThat(bucket.getDocCount(), equalTo(1L)); topHits = bucket.getAggregations().get("top-comments"); searchHits = topHits.getHits(); - assertThat(searchHits.totalHits(), equalTo(1L)); + assertThat(searchHits.getTotalHits(), equalTo(1L)); assertThat(searchHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(searchHits.getAt(0).getNestedIdentity().getOffset(), equalTo(1)); - assertThat((Integer) searchHits.getAt(0).getSource().get("date"), equalTo(4)); + assertThat((Integer) searchHits.getAt(0).getSourceAsMap().get("date"), equalTo(4)); } public void testTopHitsInSecondLayerNested() throws Exception { @@ -771,7 +771,7 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(toComments.getDocCount(), equalTo(4L)); TopHits topComments = toComments.getAggregations().get("top-comments"); - assertThat(topComments.getHits().totalHits(), equalTo(4L)); + assertThat(topComments.getHits().getTotalHits(), equalTo(4L)); assertThat(topComments.getHits().getHits().length, equalTo(4)); assertThat(topComments.getHits().getAt(0).getId(), equalTo("2")); @@ -798,53 +798,53 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(toReviewers.getDocCount(), equalTo(7L)); TopHits topReviewers = toReviewers.getAggregations().get("top-reviewers"); - assertThat(topReviewers.getHits().totalHits(), equalTo(7L)); + assertThat(topReviewers.getHits().getTotalHits(), equalTo(7L)); assertThat(topReviewers.getHits().getHits().length, equalTo(7)); assertThat(topReviewers.getHits().getAt(0).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(0).sourceAsMap().get("name"), equalTo("user a")); + assertThat((String) topReviewers.getHits().getAt(0).getSourceAsMap().get("name"), equalTo("user a")); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getChild().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(1).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(1).sourceAsMap().get("name"), equalTo("user b")); + assertThat((String) topReviewers.getHits().getAt(1).getSourceAsMap().get("name"), equalTo("user b")); assertThat(topReviewers.getHits().getAt(1).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(1).getNestedIdentity().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(1).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(1).getNestedIdentity().getChild().getOffset(), equalTo(1)); assertThat(topReviewers.getHits().getAt(2).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(2).sourceAsMap().get("name"), equalTo("user c")); + assertThat((String) topReviewers.getHits().getAt(2).getSourceAsMap().get("name"), equalTo("user c")); assertThat(topReviewers.getHits().getAt(2).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(2).getNestedIdentity().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(2).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(2).getNestedIdentity().getChild().getOffset(), equalTo(2)); assertThat(topReviewers.getHits().getAt(3).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(3).sourceAsMap().get("name"), equalTo("user c")); + assertThat((String) topReviewers.getHits().getAt(3).getSourceAsMap().get("name"), equalTo("user c")); assertThat(topReviewers.getHits().getAt(3).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(3).getNestedIdentity().getOffset(), equalTo(1)); assertThat(topReviewers.getHits().getAt(3).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(3).getNestedIdentity().getChild().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(4).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(4).sourceAsMap().get("name"), equalTo("user d")); + assertThat((String) topReviewers.getHits().getAt(4).getSourceAsMap().get("name"), equalTo("user d")); assertThat(topReviewers.getHits().getAt(4).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(4).getNestedIdentity().getOffset(), equalTo(1)); assertThat(topReviewers.getHits().getAt(4).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(4).getNestedIdentity().getChild().getOffset(), equalTo(1)); assertThat(topReviewers.getHits().getAt(5).getId(), equalTo("1")); - assertThat((String) topReviewers.getHits().getAt(5).sourceAsMap().get("name"), equalTo("user e")); + assertThat((String) topReviewers.getHits().getAt(5).getSourceAsMap().get("name"), equalTo("user e")); assertThat(topReviewers.getHits().getAt(5).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(5).getNestedIdentity().getOffset(), equalTo(1)); assertThat(topReviewers.getHits().getAt(5).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); assertThat(topReviewers.getHits().getAt(5).getNestedIdentity().getChild().getOffset(), equalTo(2)); assertThat(topReviewers.getHits().getAt(6).getId(), equalTo("2")); - assertThat((String) topReviewers.getHits().getAt(6).sourceAsMap().get("name"), equalTo("user f")); + assertThat((String) topReviewers.getHits().getAt(6).getSourceAsMap().get("name"), equalTo("user f")); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getOffset(), equalTo(0)); assertThat(topReviewers.getHits().getAt(0).getNestedIdentity().getChild().getField().string(), equalTo("reviewers")); @@ -872,7 +872,7 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(nested.getDocCount(), equalTo(4L)); SearchHits hits = ((TopHits) nested.getAggregations().get("top-comments")).getHits(); - assertThat(hits.totalHits(), equalTo(4L)); + assertThat(hits.getTotalHits(), equalTo(4L)); SearchHit searchHit = hits.getAt(0); assertThat(searchHit.getId(), equalTo("1")); assertThat(searchHit.getNestedIdentity().getField().string(), equalTo("comments")); @@ -884,14 +884,14 @@ public class TopHitsIT extends ESIntegTestCase { // Can't explain nested hit with the main query, since both are in a different scopes, also the nested doc may not even have matched with the main query // If top_hits would have a query option then we can explain that query - Explanation explanation = searchHit.explanation(); + Explanation explanation = searchHit.getExplanation(); assertFalse(explanation.isMatch()); // Returns the version of the root document. Nested docs don't have a separate version - long version = searchHit.version(); + long version = searchHit.getVersion(); assertThat(version, equalTo(1L)); - assertThat(searchHit.matchedQueries(), arrayContaining("test")); + assertThat(searchHit.getMatchedQueries(), arrayContaining("test")); SearchHitField field = searchHit.field("comments.user"); assertThat(field.getValue().toString(), equalTo("a")); @@ -899,8 +899,8 @@ public class TopHitsIT extends ESIntegTestCase { field = searchHit.field("script"); assertThat(field.getValue().toString(), equalTo("5")); - assertThat(searchHit.sourceAsMap().size(), equalTo(1)); - assertThat(XContentMapValues.extractValue("comments.message", searchHit.sourceAsMap()), equalTo("some comment")); + assertThat(searchHit.getSourceAsMap().size(), equalTo(1)); + assertThat(XContentMapValues.extractValue("comments.message", searchHit.getSourceAsMap()), equalTo("some comment")); } public void testTopHitsInNested() throws Exception { @@ -930,11 +930,11 @@ public class TopHitsIT extends ESIntegTestCase { TopHits hits = nested.getAggregations().get("comments"); SearchHits searchHits = hits.getHits(); - assertThat(searchHits.totalHits(), equalTo(numNestedDocs)); + assertThat(searchHits.getTotalHits(), equalTo(numNestedDocs)); for (int j = 0; j < 3; j++) { assertThat(searchHits.getAt(j).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(searchHits.getAt(j).getNestedIdentity().getOffset(), equalTo(0)); - assertThat((Integer) searchHits.getAt(j).sourceAsMap().get("id"), equalTo(0)); + assertThat((Integer) searchHits.getAt(j).getSourceAsMap().get("id"), equalTo(0)); HighlightField highlightField = searchHits.getAt(j).getHighlightFields().get("comments.message"); assertThat(highlightField.getFragments().length, equalTo(1)); @@ -985,12 +985,12 @@ public class TopHitsIT extends ESIntegTestCase { assertThat(bucket.getDocCount(), equalTo(10L)); TopHits topHits = bucket.getAggregations().get("hits"); SearchHits hits = topHits.getHits(); - assertThat(hits.totalHits(), equalTo(10L)); + assertThat(hits.getTotalHits(), equalTo(10L)); assertThat(hits.getHits().length, equalTo(3)); for (SearchHit hit : hits) { - assertThat(hit.source(), nullValue()); - assertThat(hit.id(), nullValue()); - assertThat(hit.type(), nullValue()); + assertThat(hit.getSourceAsMap(), nullValue()); + assertThat(hit.getId(), nullValue()); + assertThat(hit.getType(), nullValue()); } } } @@ -1067,7 +1067,7 @@ public class TopHitsIT extends ESIntegTestCase { for (Terms.Bucket bucket : terms.getBuckets()) { TopHits topHits = bucket.getAggregations().get("hits"); for (SearchHit hit : topHits.getHits().getHits()) { - assertThat(hit.score(), equalTo(4.0f)); + assertThat(hit.getScore(), equalTo(4.0f)); } } } @@ -1090,7 +1090,7 @@ public class TopHitsIT extends ESIntegTestCase { for (Terms.Bucket bucket : terms.getBuckets()) { TopHits topHits = bucket.getAggregations().get("hits"); for (SearchHit hit : topHits.getHits().getHits()) { - assertThat(hit.score(), equalTo(4.0f)); + assertThat(hit.getScore(), equalTo(4.0f)); } } } @@ -1114,7 +1114,7 @@ public class TopHitsIT extends ESIntegTestCase { for (Terms.Bucket bucket : terms.getBuckets()) { TopHits topHits = bucket.getAggregations().get("hits"); for (SearchHit hit : topHits.getHits().getHits()) { - assertThat(hit.score(), equalTo(Float.NaN)); + assertThat(hit.getScore(), equalTo(Float.NaN)); } } } @@ -1137,7 +1137,7 @@ public class TopHitsIT extends ESIntegTestCase { for (Terms.Bucket bucket : terms.getBuckets()) { TopHits topHits = bucket.getAggregations().get("hits"); for (SearchHit hit : topHits.getHits().getHits()) { - assertThat(hit.score(), equalTo(Float.NaN)); + assertThat(hit.getScore(), equalTo(Float.NaN)); } } } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java index 192dc92d4c3..2a443336383 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHitsTests.java @@ -33,8 +33,8 @@ import org.elasticsearch.common.text.Text; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.aggregations.InternalAggregationTestCase; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHits; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; import java.io.IOException; import java.util.ArrayList; @@ -69,7 +69,7 @@ public class InternalTopHitsTests extends InternalAggregationTestCase usedDocIds = new HashSet<>(); for (int i = 0; i < actualSize; i++) { float score = randomFloat(); @@ -87,12 +87,12 @@ public class InternalTopHitsTests extends InternalAggregationTestCase inputs) { - InternalSearchHits actualHits = (InternalSearchHits) reduced.getHits(); - List> allHits = new ArrayList<>(); + SearchHits actualHits = reduced.getHits(); + List> allHits = new ArrayList<>(); float maxScore = Float.MIN_VALUE; long totalHits = 0; for (int input = 0; input < inputs.size(); input++) { - InternalSearchHits internalHits = (InternalSearchHits) inputs.get(input).getHits(); - totalHits += internalHits.totalHits(); - maxScore = max(maxScore, internalHits.maxScore()); + SearchHits internalHits = inputs.get(input).getHits(); + totalHits += internalHits.getTotalHits(); + maxScore = max(maxScore, internalHits.getMaxScore()); for (int i = 0; i < internalHits.internalHits().length; i++) { ScoreDoc doc = inputs.get(input).getTopDocs().scoreDocs[i]; if (testInstancesLookSortedByField) { @@ -152,11 +152,11 @@ public class InternalTopHitsTests extends InternalAggregationTestCase scoreDocComparator() { return innerScoreDocComparator().thenComparing(s -> s.shardIndex); } - + private Comparator innerScoreDocComparator() { if (testInstancesLookSortedByField) { // Values passed to getComparator shouldn't matter diff --git a/core/src/test/java/org/elasticsearch/search/basic/SearchWhileRelocatingIT.java b/core/src/test/java/org/elasticsearch/search/basic/SearchWhileRelocatingIT.java index 54316b8d791..5908dac2feb 100644 --- a/core/src/test/java/org/elasticsearch/search/basic/SearchWhileRelocatingIT.java +++ b/core/src/test/java/org/elasticsearch/search/basic/SearchWhileRelocatingIT.java @@ -75,12 +75,12 @@ public class SearchWhileRelocatingIT extends ESIntegTestCase { try { while (!stop.get()) { SearchResponse sr = client().prepareSearch().setSize(numDocs).get(); - if (sr.getHits().totalHits() != numDocs) { + if (sr.getHits().getTotalHits() != numDocs) { // if we did not search all shards but had no failures that is potentially fine // if only the hit-count is wrong. this can happen if the cluster-state is behind when the // request comes in. It's a small window but a known limitation. if (sr.getTotalShards() != sr.getSuccessfulShards() && sr.getFailedShards() == 0) { - nonCriticalExceptions.add("Count is " + sr.getHits().totalHits() + " but " + numDocs + + nonCriticalExceptions.add("Count is " + sr.getHits().getTotalHits() + " but " + numDocs + " was expected. " + formatShardStatus(sr)); } else { assertHitCount(sr, numDocs); diff --git a/core/src/test/java/org/elasticsearch/search/basic/TransportTwoNodesSearchIT.java b/core/src/test/java/org/elasticsearch/search/basic/TransportTwoNodesSearchIT.java index 5d91159ed74..3c16ff7188c 100644 --- a/core/src/test/java/org/elasticsearch/search/basic/TransportTwoNodesSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/basic/TransportTwoNodesSearchIT.java @@ -33,7 +33,6 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; -import org.elasticsearch.search.Scroll; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.bucket.filter.Filter; @@ -44,7 +43,6 @@ import org.elasticsearch.test.ESIntegTestCase; import java.io.IOException; import java.util.Collections; -import java.util.HashSet; import java.util.Set; import java.util.TreeSet; @@ -53,7 +51,6 @@ import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH; import static org.elasticsearch.client.Requests.createIndexRequest; import static org.elasticsearch.client.Requests.searchRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; -import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; @@ -61,7 +58,6 @@ import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -139,26 +135,26 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setSearchType(DFS_QUERY_THEN_FETCH).setQuery(termQuery("multi", "test")).setSize(60).setExplain(true).setScroll(TimeValue.timeValueSeconds(30)).get(); while (true) { assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - SearchHit[] hits = searchResponse.getHits().hits(); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + SearchHit[] hits = searchResponse.getHits().getHits(); if (hits.length == 0) { break; // finished } for (int i = 0; i < hits.length; ++i) { SearchHit hit = hits[i]; - assertThat(hit.explanation(), notNullValue()); - assertThat(hit.explanation().getDetails().length, equalTo(1)); - assertThat(hit.explanation().getDetails()[0].getDetails().length, equalTo(2)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails().length, equalTo(2)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[0].getDescription(), + assertThat(hit.getExplanation(), notNullValue()); + assertThat(hit.getExplanation().getDetails().length, equalTo(1)); + assertThat(hit.getExplanation().getDetails()[0].getDetails().length, equalTo(2)); + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails().length, equalTo(2)); + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[0].getDescription(), equalTo("docFreq")); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[0].getValue(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[0].getValue(), equalTo(100.0f)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[1].getDescription(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[1].getDescription(), equalTo("docCount")); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[1].getValue(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[1].getValue(), equalTo(100.0f)); - assertThat("id[" + hit.id() + "] -> " + hit.explanation().toString(), hit.id(), equalTo(Integer.toString(100 - total - i - 1))); + assertThat("id[" + hit.getId() + "] -> " + hit.getExplanation().toString(), hit.getId(), equalTo(Integer.toString(100 - total - i - 1))); } total += hits.length; searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueSeconds(30)).get(); @@ -174,26 +170,26 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setSearchType(DFS_QUERY_THEN_FETCH).setQuery(termQuery("multi", "test")).setSize(60).setExplain(true).addSort("age", SortOrder.ASC).setScroll(TimeValue.timeValueSeconds(30)).get(); while (true) { assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - SearchHit[] hits = searchResponse.getHits().hits(); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + SearchHit[] hits = searchResponse.getHits().getHits(); if (hits.length == 0) { break; // finished } for (int i = 0; i < hits.length; ++i) { SearchHit hit = hits[i]; - assertThat(hit.explanation(), notNullValue()); - assertThat(hit.explanation().getDetails().length, equalTo(1)); - assertThat(hit.explanation().getDetails()[0].getDetails().length, equalTo(2)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails().length, equalTo(2)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[0].getDescription(), + assertThat(hit.getExplanation(), notNullValue()); + assertThat(hit.getExplanation().getDetails().length, equalTo(1)); + assertThat(hit.getExplanation().getDetails()[0].getDetails().length, equalTo(2)); + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails().length, equalTo(2)); + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[0].getDescription(), equalTo("docFreq")); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[0].getValue(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[0].getValue(), equalTo(100.0f)); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[1].getDescription(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[1].getDescription(), equalTo("docCount")); - assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDetails()[1].getValue(), + assertThat(hit.getExplanation().getDetails()[0].getDetails()[0].getDetails()[1].getValue(), equalTo(100.0f)); - assertThat("id[" + hit.id() + "]", hit.id(), equalTo(Integer.toString(total + i))); + assertThat("id[" + hit.getId() + "]", hit.getId(), equalTo(Integer.toString(total + i))); } total += hits.length; searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueSeconds(30)).get(); @@ -209,15 +205,15 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setSearchType(QUERY_THEN_FETCH).setQuery(termQuery("multi", "test")).setSize(60).setExplain(true).addSort("nid", SortOrder.DESC).setScroll(TimeValue.timeValueSeconds(30)).get(); while (true) { assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - SearchHit[] hits = searchResponse.getHits().hits(); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + SearchHit[] hits = searchResponse.getHits().getHits(); if (hits.length == 0) { break; // finished } for (int i = 0; i < hits.length; ++i) { SearchHit hit = hits[i]; - assertThat(hit.explanation(), notNullValue()); - assertThat("id[" + hit.id() + "]", hit.id(), equalTo(Integer.toString(100 - total - i - 1))); + assertThat(hit.getExplanation(), notNullValue()); + assertThat("id[" + hit.getId() + "]", hit.getId(), equalTo(Integer.toString(100 - total - i - 1))); } total += hits.length; searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueSeconds(30)).get(); @@ -237,19 +233,19 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().search(searchRequest("test").source(source.from(0).size(60)).searchType(QUERY_THEN_FETCH)).actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(60)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + assertThat(searchResponse.getHits().getHits().length, equalTo(60)); for (int i = 0; i < 60; i++) { - SearchHit hit = searchResponse.getHits().hits()[i]; - collectedIds.add(hit.id()); + SearchHit hit = searchResponse.getHits().getHits()[i]; + collectedIds.add(hit.getId()); } searchResponse = client().search(searchRequest("test").source(source.from(60).size(60)).searchType(QUERY_THEN_FETCH)).actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(40)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + assertThat(searchResponse.getHits().getHits().length, equalTo(40)); for (int i = 0; i < 40; i++) { - SearchHit hit = searchResponse.getHits().hits()[i]; - collectedIds.add(hit.id()); + SearchHit hit = searchResponse.getHits().getHits()[i]; + collectedIds.add(hit.getId()); } assertThat(collectedIds, equalTo(fullExpectedIds)); } @@ -261,15 +257,15 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setQuery(termQuery("multi", "test")).setSize(60).setExplain(true).addSort("age", SortOrder.ASC).setScroll(TimeValue.timeValueSeconds(30)).get(); while (true) { assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); - SearchHit[] hits = searchResponse.getHits().hits(); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); + SearchHit[] hits = searchResponse.getHits().getHits(); if (hits.length == 0) { break; // finished } for (int i = 0; i < hits.length; ++i) { SearchHit hit = hits[i]; - assertThat(hit.explanation(), notNullValue()); - assertThat("id[" + hit.id() + "]", hit.id(), equalTo(Integer.toString(total + i))); + assertThat(hit.getExplanation(), notNullValue()); + assertThat("id[" + hit.getId() + "]", hit.getId(), equalTo(Integer.toString(total + i))); } total += hits.length; searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueSeconds(30)).get(); @@ -290,7 +286,7 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().search(searchRequest("test").source(sourceBuilder)).actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(100L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); Global global = searchResponse.getAggregations().get("global"); Filter all = global.getAggregations().get("all"); @@ -329,22 +325,22 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { .query(termQuery("multi", "test")) .from(1000).size(20).explain(true); SearchResponse response = client().search(searchRequest("test").searchType(DFS_QUERY_THEN_FETCH).source(source)).actionGet(); - assertThat(response.getHits().hits().length, equalTo(0)); + assertThat(response.getHits().getHits().length, equalTo(0)); assertThat(response.getTotalShards(), equalTo(test.numPrimaries)); assertThat(response.getSuccessfulShards(), equalTo(test.numPrimaries)); assertThat(response.getFailedShards(), equalTo(0)); response = client().search(searchRequest("test").searchType(QUERY_THEN_FETCH).source(source)).actionGet(); assertNoFailures(response); - assertThat(response.getHits().hits().length, equalTo(0)); + assertThat(response.getHits().getHits().length, equalTo(0)); response = client().search(searchRequest("test").searchType(DFS_QUERY_THEN_FETCH).source(source)).actionGet(); assertNoFailures(response); - assertThat(response.getHits().hits().length, equalTo(0)); + assertThat(response.getHits().getHits().length, equalTo(0)); response = client().search(searchRequest("test").searchType(DFS_QUERY_THEN_FETCH).source(source)).actionGet(); assertNoFailures(response); - assertThat(response.getHits().hits().length, equalTo(0)); + assertThat(response.getHits().getHits().length, equalTo(0)); logger.info("Done Testing failed search"); } @@ -363,10 +359,10 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { assertThat(response.getResponses()[0].getFailureMessage(), notNullValue()); assertThat(response.getResponses()[1].getFailureMessage(), nullValue()); - assertThat(response.getResponses()[1].getResponse().getHits().hits().length, equalTo(1)); + assertThat(response.getResponses()[1].getResponse().getHits().getHits().length, equalTo(1)); assertThat(response.getResponses()[2].getFailureMessage(), nullValue()); - assertThat(response.getResponses()[2].getResponse().getHits().hits().length, equalTo(10)); + assertThat(response.getResponses()[2].getResponse().getHits().getHits().length, equalTo(10)); logger.info("Done Testing failed search"); } @@ -386,10 +382,10 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase { assertThat(response.getResponses()[0].getFailureMessage(), notNullValue()); assertThat(response.getResponses()[1].getFailureMessage(), nullValue()); - assertThat(response.getResponses()[1].getResponse().getHits().hits().length, equalTo(1)); + assertThat(response.getResponses()[1].getResponse().getHits().getHits().length, equalTo(1)); assertThat(response.getResponses()[2].getFailureMessage(), nullValue()); - assertThat(response.getResponses()[2].getResponse().getHits().hits().length, equalTo(10)); + assertThat(response.getResponses()[2].getResponse().getHits().getHits().length, equalTo(10)); logger.info("Done Testing failed search"); } diff --git a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java index 78cb846932f..7577b682533 100644 --- a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java @@ -140,34 +140,34 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .filter(hasChildQuery("grandchild", termQuery("gc_field", "gc_value1"), ScoreMode.None)) , ScoreMode.None))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", termQuery("p_field", "p_value1"), false))).execute() .actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1")); searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("child", termQuery("c_field", "c_value1"), false))).execute() .actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("gc1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("gc1")); searchResponse = client().prepareSearch("test").setQuery(hasParentQuery("parent", termQuery("p_field", "p_value1"), false)).execute() .actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1")); searchResponse = client().prepareSearch("test").setQuery(hasParentQuery("child", termQuery("c_field", "c_value1"), false)).execute() .actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("gc1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("gc1")); } // see #2744 @@ -185,8 +185,8 @@ public class ChildQuerySearchIT extends ESIntegTestCase { setQuery(hasChildQuery("test", matchQuery("foo", 1), ScoreMode.None)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); } @@ -209,57 +209,57 @@ public class ChildQuerySearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setQuery(idsQuery("child").addIds("c1")).storedFields("_parent").execute() .actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c1")); - assertThat(searchResponse.getHits().getAt(0).field("_parent").value().toString(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1")); + assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1")); // TEST matching on parent searchResponse = client().prepareSearch("test").setQuery(termQuery("_parent#parent", "p1")).storedFields("_parent").get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("c1"), equalTo("c2"))); - assertThat(searchResponse.getHits().getAt(0).field("_parent").value().toString(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("c1"), equalTo("c2"))); - assertThat(searchResponse.getHits().getAt(1).field("_parent").value().toString(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("c1"), equalTo("c2"))); + assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("c1"), equalTo("c2"))); + assertThat(searchResponse.getHits().getAt(1).field("_parent").getValue().toString(), equalTo("p1")); searchResponse = client().prepareSearch("test").setQuery(queryStringQuery("_parent#parent:p1")).storedFields("_parent").get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("c1"), equalTo("c2"))); - assertThat(searchResponse.getHits().getAt(0).field("_parent").value().toString(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("c1"), equalTo("c2"))); - assertThat(searchResponse.getHits().getAt(1).field("_parent").value().toString(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("c1"), equalTo("c2"))); + assertThat(searchResponse.getHits().getAt(0).field("_parent").getValue().toString(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("c1"), equalTo("c2"))); + assertThat(searchResponse.getHits().getAt(1).field("_parent").getValue().toString(), equalTo("p1")); // HAS CHILD searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "yellow")) .get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "blue")).execute() .actionGet(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2")); searchResponse = client().prepareSearch("test").setQuery(randomHasChild("child", "c_field", "red")).get(); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("p2"), equalTo("p1"))); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1"))); // HAS PARENT searchResponse = client().prepareSearch("test") .setQuery(randomHasParent("parent", "p_field", "p_value2")).get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c3")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("c4")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c3")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c4")); searchResponse = client().prepareSearch("test") .setQuery(randomHasParent("parent", "p_field", "p_value1")).get(); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("c2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c2")); } // Issue #3290 @@ -342,10 +342,10 @@ public class ChildQuerySearchIT extends ESIntegTestCase { assertNoFailures(searchResponse); Set childIds = parentToChildrenEntry.getValue(); - assertThat(searchResponse.getHits().totalHits(), equalTo((long) childIds.size())); - for (int i = 0; i < searchResponse.getHits().totalHits(); i++) { - assertThat(childIds.remove(searchResponse.getHits().getAt(i).id()), is(true)); - assertThat(searchResponse.getHits().getAt(i).score(), is(1.0f)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) childIds.size())); + for (int i = 0; i < searchResponse.getHits().getTotalHits(); i++) { + assertThat(childIds.remove(searchResponse.getHits().getAt(i).getId()), is(true)); + assertThat(searchResponse.getHits().getAt(i).getScore(), is(1.0f)); } assertThat(childIds.size(), is(0)); } @@ -378,46 +378,46 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); searchResponse = client().prepareSearch("test") .setQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p2")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2")); searchResponse = client().prepareSearch("test") .setQuery(hasChildQuery("child", termQuery("c_field", "red"), ScoreMode.None)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("p2"), equalTo("p1"))); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1"))); // HAS CHILD FILTER searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p2")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2")); searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "red"), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("p2"), equalTo("p1"))); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1"))); } public void testScopedFacet() throws Exception { @@ -443,9 +443,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { AggregationBuilders.filter("filter", boolQuery().should(termQuery("c_field", "red")).should(termQuery("c_field", "yellow"))).subAggregation( AggregationBuilders.terms("facet1").field("c_field")))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("p2"), equalTo("p1"))); - assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), anyOf(equalTo("p2"), equalTo("p1"))); + assertThat(searchResponse.getHits().getAt(1).getId(), anyOf(equalTo("p2"), equalTo("p1"))); Global global = searchResponse.getAggregations().get("global"); Filter filter = global.getAggregations().get("filter"); @@ -475,9 +475,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(0).sourceAsString(), containsString("\"p_value1\"")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\"")); // update p1 and see what that we get updated values... @@ -487,9 +487,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.None))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(0).sourceAsString(), containsString("\"p_value1_updated\"")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1_updated\"")); } public void testDfsSearchType() throws Exception { @@ -535,12 +535,12 @@ public class ChildQuerySearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", matchAllQuery(), ScoreMode.None))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", matchAllQuery(), false))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testCountApiUsage() throws Exception { @@ -589,14 +589,14 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max)) .get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), containsString("join value p1")); + assertThat(searchResponse.getHits().getAt(0).getExplanation().getDescription(), containsString("join value p1")); searchResponse = client().prepareSearch("test") .setExplain(true) .setQuery(hasParentQuery("parent", termQuery("p_field", "1"), true)) .get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().getAt(0).explanation().getDescription(), containsString("join value p1")); + assertThat(searchResponse.getHits().getAt(0).getExplanation().getDescription(), containsString("join value p1")); ExplainResponse explainResponse = client().prepareExplain("test", "parent", parentId) .setQuery(hasChildQuery("child", termQuery("c_field", "1"), ScoreMode.Max)) @@ -678,13 +678,13 @@ public class ChildQuerySearchIT extends ESIntegTestCase { fieldValueFactorFunction("c_field1")) .boostMode(CombineFunction.REPLACE), ScoreMode.Total)).get(); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("1")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(4f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(3f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("1")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(4f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(3f)); response = client() .prepareSearch("test") @@ -695,13 +695,13 @@ public class ChildQuerySearchIT extends ESIntegTestCase { fieldValueFactorFunction("c_field1")) .boostMode(CombineFunction.REPLACE), ScoreMode.Max)).get(); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(4f)); - assertThat(response.getHits().hits()[1].id(), equalTo("2")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("1")); - assertThat(response.getHits().hits()[2].score(), equalTo(2f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(4f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("1")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(2f)); response = client() .prepareSearch("test") @@ -712,13 +712,13 @@ public class ChildQuerySearchIT extends ESIntegTestCase { fieldValueFactorFunction("c_field1")) .boostMode(CombineFunction.REPLACE), ScoreMode.Avg)).get(); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(4f)); - assertThat(response.getHits().hits()[1].id(), equalTo("2")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("1")); - assertThat(response.getHits().hits()[2].score(), equalTo(1.5f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(4f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("1")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1.5f)); response = client() .prepareSearch("test") @@ -730,21 +730,21 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .boostMode(CombineFunction.REPLACE), true)) .addSort(SortBuilders.fieldSort("c_field3")).addSort(SortBuilders.scoreSort()).get(); - assertThat(response.getHits().totalHits(), equalTo(7L)); - assertThat(response.getHits().hits()[0].id(), equalTo("13")); - assertThat(response.getHits().hits()[0].score(), equalTo(5f)); - assertThat(response.getHits().hits()[1].id(), equalTo("14")); - assertThat(response.getHits().hits()[1].score(), equalTo(5f)); - assertThat(response.getHits().hits()[2].id(), equalTo("15")); - assertThat(response.getHits().hits()[2].score(), equalTo(5f)); - assertThat(response.getHits().hits()[3].id(), equalTo("16")); - assertThat(response.getHits().hits()[3].score(), equalTo(5f)); - assertThat(response.getHits().hits()[4].id(), equalTo("17")); - assertThat(response.getHits().hits()[4].score(), equalTo(5f)); - assertThat(response.getHits().hits()[5].id(), equalTo("18")); - assertThat(response.getHits().hits()[5].score(), equalTo(5f)); - assertThat(response.getHits().hits()[6].id(), equalTo("1")); - assertThat(response.getHits().hits()[6].score(), equalTo(5f)); + assertThat(response.getHits().getTotalHits(), equalTo(7L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("13")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("14")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("15")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[3].getId(), equalTo("16")); + assertThat(response.getHits().getHits()[3].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[4].getId(), equalTo("17")); + assertThat(response.getHits().getHits()[4].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[5].getId(), equalTo("18")); + assertThat(response.getHits().getHits()[5].getScore(), equalTo(5f)); + assertThat(response.getHits().getHits()[6].getId(), equalTo("1")); + assertThat(response.getHits().getHits()[6].getScore(), equalTo(5f)); } // Issue #2536 @@ -757,7 +757,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch("test") .setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get(); assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject()) .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -765,21 +765,21 @@ public class ChildQuerySearchIT extends ESIntegTestCase { response = client().prepareSearch("test") .setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get(); assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = client().prepareSearch("test").setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.Max)) .get(); assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = client().prepareSearch("test").setQuery(QueryBuilders.hasParentQuery("parent", matchQuery("text", "value"), false)).get(); assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = client().prepareSearch("test").setQuery(QueryBuilders.hasParentQuery("parent", matchQuery("text", "value"), true)) .get(); assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); } public void testHasChildAndHasParentFilter_withFilter() throws Exception { @@ -799,14 +799,14 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(boolQuery().must(matchAllQuery()).filter(hasChildQuery("child", termQuery("c_field", 1), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("1")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(matchAllQuery()).filter(hasParentQuery("parent", termQuery("p_field", 1), false))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("2")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("2")); } public void testHasChildInnerHitsHighlighting() throws Exception { @@ -826,9 +826,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .highlightQuery(QueryBuilders.matchQuery("c_field", "bar")))), false)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("1")); - SearchHit[] searchHits = searchResponse.getHits().hits()[0].getInnerHits().get("child").hits(); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); + SearchHit[] searchHits = searchResponse.getHits().getHits()[0].getInnerHits().get("child").getHits(); assertThat(searchHits.length, equalTo(1)); assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments().length, equalTo(1)); assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments()[0].string(), equalTo("foo bar")); @@ -892,23 +892,23 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .addSort("p_field", SortOrder.ASC) .setSize(5).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(10L)); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("p000")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("p001")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("p002")); - assertThat(searchResponse.getHits().hits()[3].id(), equalTo("p003")); - assertThat(searchResponse.getHits().hits()[4].id(), equalTo("p004")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(10L)); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("p000")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("p001")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("p002")); + assertThat(searchResponse.getHits().getHits()[3].getId(), equalTo("p003")); + assertThat(searchResponse.getHits().getHits()[4].getId(), equalTo("p004")); searchResponse = client().prepareSearch("test").setSearchType(searchType) .setQuery(hasParentQuery("parent", prefixQuery("p_field", "p"), true)).addSort("c_field", SortOrder.ASC) .setSize(5).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(500L)); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("c000")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("c001")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("c002")); - assertThat(searchResponse.getHits().hits()[3].id(), equalTo("c003")); - assertThat(searchResponse.getHits().hits()[4].id(), equalTo("c004")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(500L)); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("c000")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("c001")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("c002")); + assertThat(searchResponse.getHits().getHits()[3].getId(), equalTo("c003")); + assertThat(searchResponse.getHits().getHits()[4].getId(), equalTo("c004")); } } @@ -932,9 +932,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test") .setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.Total)).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(0).sourceAsString(), containsString("\"p_value1\"")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\"")); searchResponse = client() .prepareSearch("test") @@ -942,9 +942,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { boolQuery().must(matchQuery("c_field", "x")).must( hasParentQuery("parent", termQuery("p_field", "p_value2"), true))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("c3")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("c4")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("c3")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("c4")); // re-index for (int i = 0; i < 10; i++) { @@ -958,9 +958,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test").setQuery(hasChildQuery("child", termQuery("c_field", "yellow"), ScoreMode.Total)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p1")); - assertThat(searchResponse.getHits().getAt(0).sourceAsString(), containsString("\"p_value1\"")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p1")); + assertThat(searchResponse.getHits().getAt(0).getSourceAsString(), containsString("\"p_value1\"")); searchResponse = client() .prepareSearch("test") @@ -968,9 +968,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { boolQuery().must(matchQuery("c_field", "x")).must( hasParentQuery("parent", termQuery("p_field", "p_value2"), true))).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().getAt(0).id(), Matchers.anyOf(equalTo("c3"), equalTo("c4"))); - assertThat(searchResponse.getHits().getAt(1).id(), Matchers.anyOf(equalTo("c3"), equalTo("c4"))); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getId(), Matchers.anyOf(equalTo("c3"), equalTo("c4"))); + assertThat(searchResponse.getHits().getAt(1).getId(), Matchers.anyOf(equalTo("c3"), equalTo("c4"))); } // Issue #3203 @@ -993,9 +993,9 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setMinScore(3) // Score needs to be 3 or above! .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("p2")); - assertThat(searchResponse.getHits().getAt(0).score(), equalTo(3.0f)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("p2")); + assertThat(searchResponse.getHits().getAt(0).getScore(), equalTo(3.0f)); } public void testParentFieldQuery() throws Exception { @@ -1082,7 +1082,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); client().prepareIndex("test", "child", "c2").setParent("p2").setSource("c_field", "blue").get(); client().admin().indices().prepareRefresh("test").get(); @@ -1091,7 +1091,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue"), ScoreMode.None))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); } private QueryBuilder randomHasChild(String type, String field, String value) { @@ -1250,13 +1250,13 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(boolQuery().must(QueryBuilders.hasChildQuery("child", termQuery("c_field", "blue"), scoreMode)).filter(boolQuery().mustNot(termQuery("p_field", "3")))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test") .setQuery(boolQuery().must(QueryBuilders.hasChildQuery("child", termQuery("c_field", "red"), scoreMode)).filter(boolQuery().mustNot(termQuery("p_field", "3")))) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); } public void testNamedFilters() throws Exception { @@ -1384,7 +1384,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .must(QueryBuilders.hasChildQuery("child", matchQuery("c_field", "red"), ScoreMode.None)) .must(matchAllQuery()))) .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); } @@ -1397,7 +1397,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .must(matchAllQuery()))) .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testParentChildQueriesViaScrollApi() throws Exception { @@ -1429,10 +1429,10 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .actionGet(); assertNoFailures(scrollResponse); - assertThat(scrollResponse.getHits().totalHits(), equalTo(10L)); + assertThat(scrollResponse.getHits().getTotalHits(), equalTo(10L)); int scannedDocs = 0; do { - assertThat(scrollResponse.getHits().totalHits(), equalTo(10L)); + assertThat(scrollResponse.getHits().getTotalHits(), equalTo(10L)); scannedDocs += scrollResponse.getHits().getHits().length; scrollResponse = client() .prepareSearchScroll(scrollResponse.getScrollId()) @@ -1477,8 +1477,8 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setQuery(multiMatchQuery("1", "_parent#type1")) .get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); } public void testTypeIsAppliedInHasParentInnerQuery() throws Exception { @@ -1580,75 +1580,75 @@ public class ChildQuerySearchIT extends ESIntegTestCase { // Score mode = NONE response = minMaxQuery(ScoreMode.None, 0, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("2")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); - assertThat(response.getHits().hits()[2].id(), equalTo("4")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 1, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("2")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); - assertThat(response.getHits().hits()[2].id(), equalTo("4")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 2, null); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("4")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 3, null); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 4, null); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = minMaxQuery(ScoreMode.None, 0, 4); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("2")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); - assertThat(response.getHits().hits()[2].id(), equalTo("4")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 0, 3); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("2")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); - assertThat(response.getHits().hits()[2].id(), equalTo("4")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 0, 2); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("2")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.None, 2, 2); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1f)); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.None, 3, 2)); assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'")); @@ -1656,75 +1656,75 @@ public class ChildQuerySearchIT extends ESIntegTestCase { // Score mode = SUM response = minMaxQuery(ScoreMode.Total, 0, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Total, 1, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Total, 2, null); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); response = minMaxQuery(ScoreMode.Total, 3, null); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); response = minMaxQuery(ScoreMode.Total, 4, null); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = minMaxQuery(ScoreMode.Total, 0, 4); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Total, 0, 3); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(6f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(3f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(6f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Total, 0, 2); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("2")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Total, 2, 2); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Total, 3, 2)); assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'")); @@ -1732,75 +1732,75 @@ public class ChildQuerySearchIT extends ESIntegTestCase { // Score mode = MAX response = minMaxQuery(ScoreMode.Max, 0, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(2f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Max, 1, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(2f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Max, 2, null); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(2f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f)); response = minMaxQuery(ScoreMode.Max, 3, null); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); response = minMaxQuery(ScoreMode.Max, 4, null); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = minMaxQuery(ScoreMode.Max, 0, 4); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(2f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Max, 0, 3); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(3f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(2f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(3f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Max, 0, 2); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("2")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Max, 2, 2); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Max, 3, 2)); assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'")); @@ -1808,75 +1808,75 @@ public class ChildQuerySearchIT extends ESIntegTestCase { // Score mode = AVG response = minMaxQuery(ScoreMode.Avg, 0, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1.5f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Avg, 1, null); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1.5f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Avg, 2, null); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1.5f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f)); response = minMaxQuery(ScoreMode.Avg, 3, null); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); response = minMaxQuery(ScoreMode.Avg, 4, null); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); response = minMaxQuery(ScoreMode.Avg, 0, 4); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1.5f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Avg, 0, 3); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().hits()[0].id(), equalTo("4")); - assertThat(response.getHits().hits()[0].score(), equalTo(2f)); - assertThat(response.getHits().hits()[1].id(), equalTo("3")); - assertThat(response.getHits().hits()[1].score(), equalTo(1.5f)); - assertThat(response.getHits().hits()[2].id(), equalTo("2")); - assertThat(response.getHits().hits()[2].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("4")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(2f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1.5f)); + assertThat(response.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[2].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Avg, 0, 2); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(1.5f)); - assertThat(response.getHits().hits()[1].id(), equalTo("2")); - assertThat(response.getHits().hits()[1].score(), equalTo(1f)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1.5f)); + assertThat(response.getHits().getHits()[1].getId(), equalTo("2")); + assertThat(response.getHits().getHits()[1].getScore(), equalTo(1f)); response = minMaxQuery(ScoreMode.Avg, 2, 2); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().hits()[0].id(), equalTo("3")); - assertThat(response.getHits().hits()[0].score(), equalTo(1.5f)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(response.getHits().getHits()[0].getScore(), equalTo(1.5f)); e = expectThrows(IllegalArgumentException.class, () -> minMaxQuery(ScoreMode.Avg, 3, 2)); assertThat(e.getMessage(), equalTo("[has_child] 'max_children' is less than 'min_children'")); @@ -1939,7 +1939,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .highlighter(new HighlightBuilder().field(new HighlightBuilder.Field("searchText").highlighterType(highlightType))) .get(); assertHitCount(searchResponse, 1); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("parent-id")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("parent-id")); HighlightField highlightField = searchResponse.getHits().getAt(0).getHighlightFields().get("searchText"); assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown fox")); @@ -1951,7 +1951,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .highlighter(new HighlightBuilder().field(new HighlightBuilder.Field("searchText").highlighterType(highlightType))) .get(); assertHitCount(searchResponse, 1); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("child-id")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("child-id")); highlightField = searchResponse.getHits().getAt(0).getHighlightFields().get("searchText"); assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown fox")); } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java b/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java index 58170fe04a8..6c1d0877f7b 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/FetchSubPhasePluginIT.java @@ -38,7 +38,6 @@ import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.internal.InternalSearchHitField; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -130,13 +129,13 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase { if (hitContext.hit().fieldsOrNull() == null) { hitContext.hit().fields(new HashMap<>()); } - SearchHitField hitField = hitContext.hit().fields().get(NAME); + SearchHitField hitField = hitContext.hit().getFields().get(NAME); if (hitField == null) { - hitField = new InternalSearchHitField(NAME, new ArrayList<>(1)); - hitContext.hit().fields().put(NAME, hitField); + hitField = new SearchHitField(NAME, new ArrayList<>(1)); + hitContext.hit().getFields().put(NAME, hitField); } TermVectorsRequest termVectorsRequest = new TermVectorsRequest(context.indexShard().shardId().getIndex().getName(), - hitContext.hit().type(), hitContext.hit().id()); + hitContext.hit().getType(), hitContext.hit().getId()); TermVectorsResponse termVector = TermVectorsService.getTermVectors(context.indexShard(), termVectorsRequest); try { Map tv = new HashMap<>(); @@ -145,7 +144,7 @@ public class FetchSubPhasePluginIT extends ESIntegTestCase { while ((term = terms.next()) != null) { tv.put(term.utf8ToString(), terms.postings(null, PostingsEnum.ALL).freq()); } - hitField.values().add(tv); + hitField.getValues().add(tv); } catch (IOException e) { ESLoggerFactory.getLogger(FetchSubPhasePluginIT.class.getName()).info("Swallowed exception", e); } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/FetchSourceSubPhaseTests.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/FetchSourceSubPhaseTests.java index 814f514c96a..952f403b05e 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/FetchSourceSubPhaseTests.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/FetchSourceSubPhaseTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.internal.InternalSearchHit; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; @@ -45,7 +45,7 @@ public class FetchSourceSubPhaseTests extends ESTestCase { .field("field", "value") .endObject(); FetchSubPhase.HitContext hitContext = hitExecute(source, true, null, null); - assertEquals(Collections.singletonMap("field","value"), hitContext.hit().sourceAsMap()); + assertEquals(Collections.singletonMap("field","value"), hitContext.hit().getSourceAsMap()); } public void testBasicFiltering() throws IOException { @@ -54,16 +54,16 @@ public class FetchSourceSubPhaseTests extends ESTestCase { .field("field2", "value2") .endObject(); FetchSubPhase.HitContext hitContext = hitExecute(source, false, null, null); - assertNull(hitContext.hit().sourceAsMap()); + assertNull(hitContext.hit().getSourceAsMap()); hitContext = hitExecute(source, true, "field1", null); - assertEquals(Collections.singletonMap("field1","value"), hitContext.hit().sourceAsMap()); + assertEquals(Collections.singletonMap("field1","value"), hitContext.hit().getSourceAsMap()); hitContext = hitExecute(source, true, "hello", null); - assertEquals(Collections.emptyMap(), hitContext.hit().sourceAsMap()); + assertEquals(Collections.emptyMap(), hitContext.hit().getSourceAsMap()); hitContext = hitExecute(source, true, "*", "field2"); - assertEquals(Collections.singletonMap("field1","value"), hitContext.hit().sourceAsMap()); + assertEquals(Collections.singletonMap("field1","value"), hitContext.hit().getSourceAsMap()); } public void testMultipleFiltering() throws IOException { @@ -72,18 +72,18 @@ public class FetchSourceSubPhaseTests extends ESTestCase { .field("field2", "value2") .endObject(); FetchSubPhase.HitContext hitContext = hitExecuteMultiple(source, true, new String[]{"*.notexisting", "field"}, null); - assertEquals(Collections.singletonMap("field","value"), hitContext.hit().sourceAsMap()); + assertEquals(Collections.singletonMap("field","value"), hitContext.hit().getSourceAsMap()); hitContext = hitExecuteMultiple(source, true, new String[]{"field.notexisting.*", "field"}, null); - assertEquals(Collections.singletonMap("field","value"), hitContext.hit().sourceAsMap()); + assertEquals(Collections.singletonMap("field","value"), hitContext.hit().getSourceAsMap()); } public void testSourceDisabled() throws IOException { FetchSubPhase.HitContext hitContext = hitExecute(null, true, null, null); - assertNull(hitContext.hit().sourceAsMap()); + assertNull(hitContext.hit().getSourceAsMap()); hitContext = hitExecute(null, false, null, null); - assertNull(hitContext.hit().sourceAsMap()); + assertNull(hitContext.hit().getSourceAsMap()); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> hitExecute(null, true, "field1", null)); assertEquals("unable to fetch fields from _source field: _source is disabled in the mappings " + @@ -105,7 +105,7 @@ public class FetchSourceSubPhaseTests extends ESTestCase { FetchSourceContext fetchSourceContext = new FetchSourceContext(fetchSource, includes, excludes); SearchContext searchContext = new FetchSourceSubPhaseTestSearchContext(fetchSourceContext, source == null ? null : source.bytes()); FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext(); - hitContext.reset(new InternalSearchHit(1, null, null, null), null, 1, null); + hitContext.reset(new SearchHit(1, null, null, null), null, 1, null); FetchSourceSubPhase phase = new FetchSourceSubPhase(); phase.hitExecute(searchContext, hitContext); return hitContext; diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java index 1d24a1f16d5..7d5b57b8ca1 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java @@ -131,7 +131,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertSearchHit(response, 1, hasId("1")); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(2L)); + assertThat(innerHits.getTotalHits(), equalTo(2L)); assertThat(innerHits.getHits().length, equalTo(2)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); @@ -150,7 +150,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getShard(), notNullValue()); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(3L)); + assertThat(innerHits.getTotalHits(), equalTo(3L)); assertThat(innerHits.getHits().length, equalTo(3)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); @@ -177,7 +177,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getHighlightFields().get("comments.message").getFragments()[0].string(), equalTo("fox eat quick")); - assertThat(innerHits.getAt(0).explanation().toString(), containsString("weight(comments.message:fox in")); + assertThat(innerHits.getAt(0).getExplanation().toString(), containsString("weight(comments.message:fox in")); assertThat(innerHits.getAt(0).getFields().get("comments.message").getValue().toString(), equalTo("eat")); assertThat(innerHits.getAt(0).getFields().get("script").getValue().toString(), equalTo("5")); } @@ -224,7 +224,7 @@ public class InnerHitsIT extends ESIntegTestCase { SearchHit searchHit = searchResponse.getHits().getAt(i); assertThat(searchHit.getShard(), notNullValue()); SearchHits inner = searchHit.getInnerHits().get("a"); - assertThat(inner.totalHits(), equalTo((long) field1InnerObjects[i])); + assertThat(inner.getTotalHits(), equalTo((long) field1InnerObjects[i])); for (int j = 0; j < field1InnerObjects[i] && j < size; j++) { SearchHit innerHit = inner.getAt(j); assertThat(innerHit.getNestedIdentity().getField().string(), equalTo("field1")); @@ -233,7 +233,7 @@ public class InnerHitsIT extends ESIntegTestCase { } inner = searchHit.getInnerHits().get("b"); - assertThat(inner.totalHits(), equalTo((long) field2InnerObjects[i])); + assertThat(inner.getTotalHits(), equalTo((long) field2InnerObjects[i])); for (int j = 0; j < field2InnerObjects[i] && j < size; j++) { SearchHit innerHit = inner.getAt(j); assertThat(innerHit.getNestedIdentity().getField().string(), equalTo("field2")); @@ -271,12 +271,12 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(2L)); + assertThat(innerHits.getTotalHits(), equalTo(2L)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).type(), equalTo("comment")); + assertThat(innerHits.getAt(0).getType(), equalTo("comment")); assertThat(innerHits.getAt(1).getId(), equalTo("2")); - assertThat(innerHits.getAt(1).type(), equalTo("comment")); + assertThat(innerHits.getAt(1).getType(), equalTo("comment")); response = client().prepareSearch("articles") .setQuery(hasChildQuery("comment", matchQuery("message", "elephant"), ScoreMode.None) @@ -288,14 +288,14 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(3L)); + assertThat(innerHits.getTotalHits(), equalTo(3L)); assertThat(innerHits.getAt(0).getId(), equalTo("4")); - assertThat(innerHits.getAt(0).type(), equalTo("comment")); + assertThat(innerHits.getAt(0).getType(), equalTo("comment")); assertThat(innerHits.getAt(1).getId(), equalTo("5")); - assertThat(innerHits.getAt(1).type(), equalTo("comment")); + assertThat(innerHits.getAt(1).getType(), equalTo("comment")); assertThat(innerHits.getAt(2).getId(), equalTo("6")); - assertThat(innerHits.getAt(2).type(), equalTo("comment")); + assertThat(innerHits.getAt(2).getType(), equalTo("comment")); response = client().prepareSearch("articles") .setQuery( @@ -312,7 +312,7 @@ public class InnerHitsIT extends ESIntegTestCase { innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getHighlightFields().get("message").getFragments()[0].string(), equalTo("fox eat quick")); - assertThat(innerHits.getAt(0).explanation().toString(), containsString("weight(message:fox")); + assertThat(innerHits.getAt(0).getExplanation().toString(), containsString("weight(message:fox")); assertThat(innerHits.getAt(0).getFields().get("message").getValue().toString(), equalTo("eat")); assertThat(innerHits.getAt(0).getFields().get("script").getValue().toString(), equalTo("5")); } @@ -377,7 +377,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(searchHit.getShard(), notNullValue()); SearchHits inner = searchHit.getInnerHits().get("a"); - assertThat(inner.totalHits(), equalTo((long) child1InnerObjects[parent])); + assertThat(inner.getTotalHits(), equalTo((long) child1InnerObjects[parent])); for (int child = 0; child < child1InnerObjects[parent] && child < size; child++) { SearchHit innerHit = inner.getAt(child); assertThat(innerHit.getType(), equalTo("child1")); @@ -388,7 +388,7 @@ public class InnerHitsIT extends ESIntegTestCase { offset1 += child1InnerObjects[parent]; inner = searchHit.getInnerHits().get("b"); - assertThat(inner.totalHits(), equalTo((long) child2InnerObjects[parent])); + assertThat(inner.getTotalHits(), equalTo((long) child2InnerObjects[parent])); for (int child = 0; child < child2InnerObjects[parent] && child < size; child++) { SearchHit innerHit = inner.getAt(child); assertThat(innerHit.getType(), equalTo("child2")); @@ -432,14 +432,14 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(searchHit.getType(), equalTo("answer")); assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L)); assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question")); - assertThat(searchHit.getInnerHits().get("question").getAt(0).id(), equalTo("1")); + assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("1")); searchHit = response.getHits().getAt(1); assertThat(searchHit.getId(), equalTo("2")); assertThat(searchHit.getType(), equalTo("answer")); assertThat(searchHit.getInnerHits().get("question").getTotalHits(), equalTo(1L)); assertThat(searchHit.getInnerHits().get("question").getAt(0).getType(), equalTo("question")); - assertThat(searchHit.getInnerHits().get("question").getAt(0).id(), equalTo("2")); + assertThat(searchHit.getInnerHits().get("question").getAt(0).getId(), equalTo("2")); } public void testParentChildMultipleLayers() throws Exception { @@ -470,14 +470,14 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).type(), equalTo("comment")); + assertThat(innerHits.getAt(0).getType(), equalTo("comment")); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); - assertThat(innerHits.getAt(0).type(), equalTo("remark")); + assertThat(innerHits.getAt(0).getType(), equalTo("remark")); response = client().prepareSearch("articles") .setQuery(hasChildQuery("comment", @@ -491,14 +491,14 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); - assertThat(innerHits.getAt(0).type(), equalTo("comment")); + assertThat(innerHits.getAt(0).getType(), equalTo("comment")); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); - assertThat(innerHits.getAt(0).type(), equalTo("remark")); + assertThat(innerHits.getAt(0).getType(), equalTo("remark")); } public void testNestedMultipleLayers() throws Exception { @@ -556,13 +556,13 @@ public class InnerHitsIT extends ESIntegTestCase { assertSearchHit(response, 1, hasId("1")); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comments"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(innerHits.getAt(0).getNestedIdentity().getOffset(), equalTo(0)); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getId(), equalTo("1")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); @@ -579,7 +579,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertSearchHit(response, 1, hasId("2")); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comments.remarks"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); @@ -599,13 +599,13 @@ public class InnerHitsIT extends ESIntegTestCase { assertSearchHit(response, 1, hasId("2")); assertThat(response.getHits().getAt(0).getInnerHits().size(), equalTo(1)); innerHits = response.getHits().getAt(0).getInnerHits().get("comments"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(innerHits.getAt(0).getNestedIdentity().getOffset(), equalTo(0)); innerHits = innerHits.getAt(0).getInnerHits().get("remark"); - assertThat(innerHits.totalHits(), equalTo(1L)); + assertThat(innerHits.getTotalHits(), equalTo(1L)); assertThat(innerHits.getHits().length, equalTo(1)); assertThat(innerHits.getAt(0).getId(), equalTo("2")); assertThat(innerHits.getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); @@ -631,9 +631,9 @@ public class InnerHitsIT extends ESIntegTestCase { .get(); assertNoFailures(response); assertHitCount(response, 1); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getTotalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getId(), equalTo("1")); assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getField().string(), equalTo("comments")); assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getOffset(), equalTo(0)); @@ -673,10 +673,10 @@ public class InnerHitsIT extends ESIntegTestCase { assertNoFailures(response); assertHitCount(response, 1); SearchHit hit = response.getHits().getAt(0); - assertThat(hit.id(), equalTo("1")); + assertThat(hit.getId(), equalTo("1")); SearchHits messages = hit.getInnerHits().get("comments.messages"); assertThat(messages.getTotalHits(), equalTo(1L)); - assertThat(messages.getAt(0).id(), equalTo("1")); + assertThat(messages.getAt(0).getId(), equalTo("1")); assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages")); assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(0)); assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue()); @@ -687,10 +687,10 @@ public class InnerHitsIT extends ESIntegTestCase { assertNoFailures(response); assertHitCount(response, 1); hit = response.getHits().getAt(0); - assertThat(hit.id(), equalTo("1")); + assertThat(hit.getId(), equalTo("1")); messages = hit.getInnerHits().get("comments.messages"); assertThat(messages.getTotalHits(), equalTo(1L)); - assertThat(messages.getAt(0).id(), equalTo("1")); + assertThat(messages.getAt(0).getId(), equalTo("1")); assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages")); assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(1)); assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue()); @@ -708,10 +708,10 @@ public class InnerHitsIT extends ESIntegTestCase { assertNoFailures(response); assertHitCount(response, 1); hit = response.getHits().getAt(0);; - assertThat(hit.id(), equalTo("1")); + assertThat(hit.getId(), equalTo("1")); messages = hit.getInnerHits().get("comments.messages"); assertThat(messages.getTotalHits(), equalTo(1L)); - assertThat(messages.getAt(0).id(), equalTo("1")); + assertThat(messages.getAt(0).getId(), equalTo("1")); assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages")); assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(0)); assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue()); @@ -778,19 +778,19 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(innerHits.getAt(3).getId(), equalTo("earl4")); SearchHits innerInnerHits = innerHits.getAt(0).getInnerHits().get("barons"); - assertThat(innerInnerHits.totalHits(), equalTo(1L)); + assertThat(innerInnerHits.getTotalHits(), equalTo(1L)); assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron1")); innerInnerHits = innerHits.getAt(1).getInnerHits().get("barons"); - assertThat(innerInnerHits.totalHits(), equalTo(1L)); + assertThat(innerInnerHits.getTotalHits(), equalTo(1L)); assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron2")); innerInnerHits = innerHits.getAt(2).getInnerHits().get("barons"); - assertThat(innerInnerHits.totalHits(), equalTo(1L)); + assertThat(innerInnerHits.getTotalHits(), equalTo(1L)); assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron3")); innerInnerHits = innerHits.getAt(3).getInnerHits().get("barons"); - assertThat(innerInnerHits.totalHits(), equalTo(1L)); + assertThat(innerInnerHits.getTotalHits(), equalTo(1L)); assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron4")); innerHits = response.getHits().getAt(0).getInnerHits().get("princes"); @@ -798,7 +798,7 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(innerHits.getAt(0).getId(), equalTo("prince")); innerInnerHits = innerHits.getAt(0).getInnerHits().get("kings"); - assertThat(innerInnerHits.totalHits(), equalTo(1L)); + assertThat(innerInnerHits.getTotalHits(), equalTo(1L)); assertThat(innerInnerHits.getAt(0).getId(), equalTo("king")); } @@ -884,8 +884,8 @@ public class InnerHitsIT extends ESIntegTestCase { .get(); assertNoFailures(searchResponse); assertAllSuccessful(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo((long) numDocs)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("0")); + assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) numDocs)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("0")); assertThat(searchResponse.getHits().getAt(0).getInnerHits().get("nested1").getTotalHits(), equalTo(2L)); assertThat(searchResponse.getHits().getAt(0).getInnerHits().get("nested1").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(searchResponse.getHits().getAt(0).getInnerHits().get("nested1").getAt(0).getMatchedQueries()[0], equalTo("test1")); @@ -893,13 +893,13 @@ public class InnerHitsIT extends ESIntegTestCase { assertThat(searchResponse.getHits().getAt(0).getInnerHits().get("nested1").getAt(1).getMatchedQueries()[0], equalTo("test3")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("1")); assertThat(searchResponse.getHits().getAt(1).getInnerHits().get("nested1").getTotalHits(), equalTo(1L)); assertThat(searchResponse.getHits().getAt(1).getInnerHits().get("nested1").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(searchResponse.getHits().getAt(1).getInnerHits().get("nested1").getAt(0).getMatchedQueries()[0], equalTo("test2")); for (int i = 2; i < numDocs; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(String.valueOf(i))); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(String.valueOf(i))); assertThat(searchResponse.getHits().getAt(i).getInnerHits().get("nested1").getTotalHits(), equalTo(1L)); assertThat(searchResponse.getHits().getAt(i).getInnerHits().get("nested1").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(searchResponse.getHits().getAt(i).getInnerHits().get("nested1").getAt(0).getMatchedQueries()[0], equalTo("test3")); @@ -922,12 +922,12 @@ public class InnerHitsIT extends ESIntegTestCase { .addSort("_uid", SortOrder.ASC) .get(); assertHitCount(response, 2); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getTotalHits(), equalTo(1L)); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name1")); - assertThat(response.getHits().getAt(1).id(), equalTo("2")); + assertThat(response.getHits().getAt(1).getId(), equalTo("2")); assertThat(response.getHits().getAt(1).getInnerHits().get("child").getTotalHits(), equalTo(1L)); assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name1")); @@ -939,7 +939,7 @@ public class InnerHitsIT extends ESIntegTestCase { .addSort("_uid", SortOrder.ASC) .get(); assertHitCount(response, 1); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getTotalHits(), equalTo(1L)); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1)); assertThat(response.getHits().getAt(0).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name2")); @@ -1004,10 +1004,10 @@ public class InnerHitsIT extends ESIntegTestCase { assertNoFailures(response); assertHitCount(response, 1); - assertThat(response.getHits().getAt(0).getInnerHits().get("comments").totalHits(), equalTo(2L)); - assertThat(extractValue("comments.message", response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).sourceAsMap()), + assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getTotalHits(), equalTo(2L)); + assertThat(extractValue("comments.message", response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getSourceAsMap()), equalTo("fox eat quick")); - assertThat(extractValue("comments.message", response.getHits().getAt(0).getInnerHits().get("comments").getAt(1).sourceAsMap()), + assertThat(extractValue("comments.message", response.getHits().getAt(0).getInnerHits().get("comments").getAt(1).getSourceAsMap()), equalTo("fox ate rabbit x y z")); } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesIT.java index 12d9f77c603..75f61f65320 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/MatchedQueriesIT.java @@ -54,14 +54,14 @@ public class MatchedQueriesIT extends ESIntegTestCase { .get(); assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("3") || hit.id().equals("2")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("test2")); - } else if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("test1")); + if (hit.getId().equals("3") || hit.getId().equals("2")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("test2")); + } else if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("test1")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } @@ -71,14 +71,14 @@ public class MatchedQueriesIT extends ESIntegTestCase { .should(rangeQuery("number").gt(2).queryName("test2"))).get(); assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1") || hit.id().equals("2")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("test1")); - } else if (hit.id().equals("3")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("test2")); + if (hit.getId().equals("1") || hit.getId().equals("2")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("test1")); + } else if (hit.getId().equals("3")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("test2")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -99,15 +99,15 @@ public class MatchedQueriesIT extends ESIntegTestCase { termQuery("title", "title1").queryName("title"))).get(); assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(2)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); - assertThat(hit.matchedQueries(), hasItemInArray("title")); - } else if (hit.id().equals("2") || hit.id().equals("3")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(2)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); + assertThat(hit.getMatchedQueries(), hasItemInArray("title")); + } else if (hit.getId().equals("2") || hit.getId().equals("3")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } @@ -119,15 +119,15 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(2)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); - assertThat(hit.matchedQueries(), hasItemInArray("title")); - } else if (hit.id().equals("2") || hit.id().equals("3")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(2)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); + assertThat(hit.getMatchedQueries(), hasItemInArray("title")); + } else if (hit.getId().equals("2") || hit.getId().equals("3")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -146,12 +146,12 @@ public class MatchedQueriesIT extends ESIntegTestCase { .setPostFilter(termQuery("name", "test").queryName("name")).get(); assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1") || hit.id().equals("2") || hit.id().equals("3")) { - assertThat(hit.matchedQueries().length, equalTo(2)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); - assertThat(hit.matchedQueries(), hasItemInArray("title")); + if (hit.getId().equals("1") || hit.getId().equals("2") || hit.getId().equals("3")) { + assertThat(hit.getMatchedQueries().length, equalTo(2)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); + assertThat(hit.getMatchedQueries(), hasItemInArray("title")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } @@ -160,12 +160,12 @@ public class MatchedQueriesIT extends ESIntegTestCase { .setPostFilter(matchQuery("name", "test").queryName("name")).get(); assertHitCount(searchResponse, 3L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1") || hit.id().equals("2") || hit.id().equals("3")) { - assertThat(hit.matchedQueries().length, equalTo(2)); - assertThat(hit.matchedQueries(), hasItemInArray("name")); - assertThat(hit.matchedQueries(), hasItemInArray("title")); + if (hit.getId().equals("1") || hit.getId().equals("2") || hit.getId().equals("3")) { + assertThat(hit.getMatchedQueries().length, equalTo(2)); + assertThat(hit.getMatchedQueries(), hasItemInArray("name")); + assertThat(hit.getMatchedQueries(), hasItemInArray("title")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -182,11 +182,11 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 1L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("regex")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("regex")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -203,11 +203,11 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 1L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("prefix")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("prefix")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -224,11 +224,11 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 1L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("fuzzy")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("fuzzy")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -245,11 +245,11 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 1L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("wildcard")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("wildcard")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -266,11 +266,11 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 1L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("span")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("span")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } @@ -301,14 +301,14 @@ public class MatchedQueriesIT extends ESIntegTestCase { assertHitCount(searchResponse, 2L); for (SearchHit hit : searchResponse.getHits()) { - if (hit.id().equals("1")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("dolor")); - } else if (hit.id().equals("2")) { - assertThat(hit.matchedQueries().length, equalTo(1)); - assertThat(hit.matchedQueries(), hasItemInArray("elit")); + if (hit.getId().equals("1")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("dolor")); + } else if (hit.getId().equals("2")) { + assertThat(hit.getMatchedQueries().length, equalTo(1)); + assertThat(hit.getMatchedQueries(), hasItemInArray("elit")); } else { - fail("Unexpected document returned with id " + hit.id()); + fail("Unexpected document returned with id " + hit.getId()); } } } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java index ca7548257ef..27b0399944d 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java @@ -972,7 +972,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { for (int i = 0; i < COUNT; i++) { SearchHit hit = searchResponse.getHits().getHits()[i]; // LUCENE 3.1 UPGRADE: Caused adding the space at the end... - assertHighlight(searchResponse, i, "field1", 0, 1, equalTo("test " + hit.id())); + assertHighlight(searchResponse, i, "field1", 0, 1, equalTo("test " + hit.getId())); } } @@ -1599,7 +1599,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { .highlighter( new HighlightBuilder().field(new HighlightBuilder.Field("highlight_field").fragmentSize(-1).numOfFragments(1) .fragmenter("simple")).highlighterType(type)).get(); - assertThat(response.getHits().hits()[0].highlightFields().isEmpty(), equalTo(true)); + assertThat(response.getHits().getHits()[0].getHighlightFields().isEmpty(), equalTo(true)); } } @@ -2151,11 +2151,11 @@ public class HighlighterSearchIT extends ESIntegTestCase { assertHitCount(searchResponse, 2L); for (SearchHit searchHit : searchResponse.getHits()) { - if ("1".equals(searchHit.id())) { + if ("1".equals(searchHit.getId())) { assertHighlight(searchHit, "field1", 0, 1, equalTo("The quick brown fox jumps over the lazy dog. " + "The lazy red fox jumps over the quick dog. " + "The quick brown dog jumps over the lazy fox.")); - } else if ("2".equals(searchHit.id())) { + } else if ("2".equals(searchHit.getId())) { assertHighlight(searchHit, "field1", 0, 3, equalTo("The quick brown fox jumps over the lazy dog. Second sentence not finished")); assertHighlight(searchHit, "field1", 1, 3, equalTo("The lazy red fox jumps over the quick dog.")); @@ -2244,7 +2244,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet(); - Map highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields(); + Map highlightFieldMap = searchResponse.getHits().getAt(0).getHighlightFields(); assertThat(highlightFieldMap.size(), equalTo(1)); HighlightField field1 = highlightFieldMap.get("field1"); assertThat(field1.fragments().length, equalTo(5)); @@ -2663,9 +2663,9 @@ public class HighlighterSearchIT extends ESIntegTestCase { SearchResponse searchResponse = searchRequestBuilder.get(); assertHitCount(searchResponse, COUNT); - assertThat(searchResponse.getHits().hits().length, equalTo(COUNT)); + assertThat(searchResponse.getHits().getHits().length, equalTo(COUNT)); for (SearchHit hit : searchResponse.getHits()) { - String prefix = prefixes.get(hit.id()); + String prefix = prefixes.get(hit.getId()); assertHighlight(hit, "field1", 0, 1, equalTo("Sentence " + prefix + " test.")); } } @@ -2817,8 +2817,8 @@ public class HighlighterSearchIT extends ESIntegTestCase { new SearchSourceBuilder().query(query) .highlighter(new HighlightBuilder().field("*").highlighterType(highlighterType))).get(); assertNoFailures(search); - assertThat(search.getHits().totalHits(), equalTo(1L)); - assertThat(search.getHits().getAt(0).highlightFields().get("text").fragments().length, equalTo(1)); + assertThat(search.getHits().getTotalHits(), equalTo(1L)); + assertThat(search.getHits().getAt(0).getHighlightFields().get("text").fragments().length, equalTo(1)); } public void testGeoFieldHighlightingWhenQueryGetsRewritten() throws IOException { @@ -2853,7 +2853,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { SearchResponse search = client().prepareSearch().setSource( new SearchSourceBuilder().query(query).highlighter(new HighlightBuilder().highlighterType("plain").field("jd"))).get(); assertNoFailures(search); - assertThat(search.getHits().totalHits(), equalTo(1L)); + assertThat(search.getHits().getTotalHits(), equalTo(1L)); } @@ -2880,7 +2880,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { .query(QueryBuilders.matchQuery("keyword_field", "some text")) .highlighter(new HighlightBuilder().field("*"))).get(); assertNoFailures(search); - assertThat(search.getHits().totalHits(), equalTo(1L)); + assertThat(search.getHits().getTotalHits(), equalTo(1L)); assertThat(search.getHits().getAt(0).getHighlightFields().get("keyword_field").getFragments()[0].string(), equalTo("some text")); } @@ -2918,7 +2918,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { .requireFieldMatch(false)) .get(); assertHitCount(searchResponse, 1); - HighlightField field = searchResponse.getHits().getAt(0).highlightFields().get("foo_text"); + HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo_text"); assertThat(field.getFragments().length, equalTo(2)); assertThat(field.getFragments()[0].string(), equalTo("brown")); assertThat(field.getFragments()[1].string(), equalTo("cow")); @@ -2937,7 +2937,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { .field(new Field("text")).highlighterType(type)) .get(); assertHitCount(searchResponse, 1); - HighlightField field = searchResponse.getHits().getAt(0).highlightFields().get("text"); + HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("text"); assertThat(field.getFragments().length, equalTo(1)); assertThat(field.getFragments()[0].string(), equalTo("brown")); } @@ -2960,7 +2960,7 @@ public class HighlighterSearchIT extends ESIntegTestCase { .field(new Field("text")).highlighterType(type)) .get(); assertHitCount(searchResponse, 1); - HighlightField field = searchResponse.getHits().getAt(0).highlightFields().get("text"); + HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("text"); assertThat(field.getFragments().length, equalTo(1)); assertThat(field.getFragments()[0].string(), equalTo("brown")); } diff --git a/core/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java b/core/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java index a089fc3dd51..62ef56f2dd7 100644 --- a/core/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java +++ b/core/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java @@ -177,28 +177,28 @@ public class SearchFieldsIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("field1").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field1").value().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field1").getValue().toString(), equalTo("value1")); // field2 is not stored, check that it is not extracted from source. searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("field2").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(0)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field2"), nullValue()); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(0)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field2"), nullValue()); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("field3").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("*3").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); searchResponse = client().prepareSearch() @@ -208,32 +208,32 @@ public class SearchFieldsIT extends ESIntegTestCase { .addStoredField("field2") .get(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(2)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); - assertThat(searchResponse.getHits().getAt(0).fields().get("field1").value().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(2)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field1").getValue().toString(), equalTo("value1")); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("field*").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(2)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); - assertThat(searchResponse.getHits().getAt(0).fields().get("field1").value().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(2)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field1").getValue().toString(), equalTo("value1")); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("f*3").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).addStoredField("*").execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).source(), nullValue()); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(2)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field1").value().toString(), equalTo("value1")); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getSourceAsMap(), nullValue()); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(2)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field1").getValue().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -241,11 +241,11 @@ public class SearchFieldsIT extends ESIntegTestCase { .addStoredField("_source") .get(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).source(), notNullValue()); - assertThat(searchResponse.getHits().getAt(0).fields().size(), equalTo(2)); - assertThat(searchResponse.getHits().getAt(0).fields().get("field1").value().toString(), equalTo("value1")); - assertThat(searchResponse.getHits().getAt(0).fields().get("field3").value().toString(), equalTo("value3")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getSourceAsMap(), notNullValue()); + assertThat(searchResponse.getHits().getAt(0).getFields().size(), equalTo(2)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field1").getValue().toString(), equalTo("value1")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("field3").getValue().toString(), equalTo("value3")); } public void testScriptDocAndFields() throws Exception { @@ -296,26 +296,26 @@ public class SearchFieldsIT extends ESIntegTestCase { assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo(3L)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); assertFalse(response.getHits().getAt(0).hasSource()); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); - Set fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); + Set fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(newHashSet("sNum1", "sNum1_field", "date1"))); - assertThat(response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(1.0)); - assertThat(response.getHits().getAt(0).fields().get("sNum1_field").values().get(0), equalTo(1.0)); - assertThat(response.getHits().getAt(0).fields().get("date1").values().get(0), equalTo(0L)); - assertThat(response.getHits().getAt(1).id(), equalTo("2")); - fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getAt(0).getFields().get("sNum1").getValues().get(0), equalTo(1.0)); + assertThat(response.getHits().getAt(0).getFields().get("sNum1_field").getValues().get(0), equalTo(1.0)); + assertThat(response.getHits().getAt(0).getFields().get("date1").getValues().get(0), equalTo(0L)); + assertThat(response.getHits().getAt(1).getId(), equalTo("2")); + fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(newHashSet("sNum1", "sNum1_field", "date1"))); - assertThat(response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(2.0)); - assertThat(response.getHits().getAt(1).fields().get("sNum1_field").values().get(0), equalTo(2.0)); - assertThat(response.getHits().getAt(1).fields().get("date1").values().get(0), equalTo(25000L)); - assertThat(response.getHits().getAt(2).id(), equalTo("3")); - fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getAt(1).getFields().get("sNum1").getValues().get(0), equalTo(2.0)); + assertThat(response.getHits().getAt(1).getFields().get("sNum1_field").getValues().get(0), equalTo(2.0)); + assertThat(response.getHits().getAt(1).getFields().get("date1").getValues().get(0), equalTo(25000L)); + assertThat(response.getHits().getAt(2).getId(), equalTo("3")); + fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(newHashSet("sNum1", "sNum1_field", "date1"))); - assertThat(response.getHits().getAt(2).fields().get("sNum1").values().get(0), equalTo(3.0)); - assertThat(response.getHits().getAt(2).fields().get("sNum1_field").values().get(0), equalTo(3.0)); - assertThat(response.getHits().getAt(2).fields().get("date1").values().get(0), equalTo(120000L)); + assertThat(response.getHits().getAt(2).getFields().get("sNum1").getValues().get(0), equalTo(3.0)); + assertThat(response.getHits().getAt(2).getFields().get("sNum1_field").getValues().get(0), equalTo(3.0)); + assertThat(response.getHits().getAt(2).getFields().get("date1").getValues().get(0), equalTo(120000L)); logger.info("running doc['num1'].value * factor"); Map params = MapBuilder.newMapBuilder().put("factor", 2.0).map(); @@ -325,19 +325,19 @@ public class SearchFieldsIT extends ESIntegTestCase { .addScriptField("sNum1", new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['num1'].value * factor", params)) .get(); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); - fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); + fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(singleton("sNum1"))); - assertThat(response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(2.0)); - assertThat(response.getHits().getAt(1).id(), equalTo("2")); - fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getAt(0).getFields().get("sNum1").getValues().get(0), equalTo(2.0)); + assertThat(response.getHits().getAt(1).getId(), equalTo("2")); + fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(singleton("sNum1"))); - assertThat(response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(4.0)); - assertThat(response.getHits().getAt(2).id(), equalTo("3")); - fields = new HashSet<>(response.getHits().getAt(0).fields().keySet()); + assertThat(response.getHits().getAt(1).getFields().get("sNum1").getValues().get(0), equalTo(4.0)); + assertThat(response.getHits().getAt(2).getId(), equalTo("3")); + fields = new HashSet<>(response.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(singleton("sNum1"))); - assertThat(response.getHits().getAt(2).fields().get("sNum1").values().get(0), equalTo(6.0)); + assertThat(response.getHits().getAt(2).getFields().get("sNum1").getValues().get(0), equalTo(6.0)); } public void testUidBasedScriptFields() throws Exception { @@ -360,12 +360,12 @@ public class SearchFieldsIT extends ESIntegTestCase { assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo((long)numDocs)); + assertThat(response.getHits().getTotalHits(), equalTo((long)numDocs)); for (int i = 0; i < numDocs; i++) { - assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - Set fields = new HashSet<>(response.getHits().getAt(i).fields().keySet()); + assertThat(response.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + Set fields = new HashSet<>(response.getHits().getAt(i).getFields().keySet()); assertThat(fields, equalTo(singleton("uid"))); - assertThat(response.getHits().getAt(i).fields().get("uid").value(), equalTo("type1#" + Integer.toString(i))); + assertThat(response.getHits().getAt(i).getFields().get("uid").getValue(), equalTo("type1#" + Integer.toString(i))); } response = client().prepareSearch() @@ -377,12 +377,12 @@ public class SearchFieldsIT extends ESIntegTestCase { assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo((long)numDocs)); + assertThat(response.getHits().getTotalHits(), equalTo((long)numDocs)); for (int i = 0; i < numDocs; i++) { - assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - Set fields = new HashSet<>(response.getHits().getAt(i).fields().keySet()); + assertThat(response.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + Set fields = new HashSet<>(response.getHits().getAt(i).getFields().keySet()); assertThat(fields, equalTo(singleton("id"))); - assertThat(response.getHits().getAt(i).fields().get("id").value(), equalTo(Integer.toString(i))); + assertThat(response.getHits().getAt(i).getFields().get("id").getValue(), equalTo(Integer.toString(i))); } response = client().prepareSearch() @@ -395,12 +395,12 @@ public class SearchFieldsIT extends ESIntegTestCase { assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo((long)numDocs)); + assertThat(response.getHits().getTotalHits(), equalTo((long)numDocs)); for (int i = 0; i < numDocs; i++) { - assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - Set fields = new HashSet<>(response.getHits().getAt(i).fields().keySet()); + assertThat(response.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + Set fields = new HashSet<>(response.getHits().getAt(i).getFields().keySet()); assertThat(fields, equalTo(singleton("type"))); - assertThat(response.getHits().getAt(i).fields().get("type").value(), equalTo("type1")); + assertThat(response.getHits().getAt(i).getFields().get("type").getValue(), equalTo("type1")); } response = client().prepareSearch() @@ -415,14 +415,14 @@ public class SearchFieldsIT extends ESIntegTestCase { assertNoFailures(response); - assertThat(response.getHits().totalHits(), equalTo((long)numDocs)); + assertThat(response.getHits().getTotalHits(), equalTo((long)numDocs)); for (int i = 0; i < numDocs; i++) { - assertThat(response.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - Set fields = new HashSet<>(response.getHits().getAt(i).fields().keySet()); + assertThat(response.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + Set fields = new HashSet<>(response.getHits().getAt(i).getFields().keySet()); assertThat(fields, equalTo(newHashSet("uid", "type", "id"))); - assertThat(response.getHits().getAt(i).fields().get("uid").value(), equalTo("type1#" + Integer.toString(i))); - assertThat(response.getHits().getAt(i).fields().get("type").value(), equalTo("type1")); - assertThat(response.getHits().getAt(i).fields().get("id").value(), equalTo(Integer.toString(i))); + assertThat(response.getHits().getAt(i).getFields().get("uid").getValue(), equalTo("type1#" + Integer.toString(i))); + assertThat(response.getHits().getAt(i).getFields().get("type").getValue(), equalTo("type1")); + assertThat(response.getHits().getAt(i).getFields().get("id").getValue(), equalTo(Integer.toString(i))); } } @@ -451,24 +451,24 @@ public class SearchFieldsIT extends ESIntegTestCase { assertThat("Failures " + Arrays.toString(response.getShardFailures()), response.getShardFailures().length, equalTo(0)); - assertThat(response.getHits().getAt(0).field("s_obj1_test").value().toString(), equalTo("something")); + assertThat(response.getHits().getAt(0).field("s_obj1_test").getValue().toString(), equalTo("something")); - Map sObj1 = response.getHits().getAt(0).field("s_obj1").value(); + Map sObj1 = response.getHits().getAt(0).field("s_obj1").getValue(); assertThat(sObj1.get("test").toString(), equalTo("something")); - assertThat(response.getHits().getAt(0).field("s_obj1_test").value().toString(), equalTo("something")); + assertThat(response.getHits().getAt(0).field("s_obj1_test").getValue().toString(), equalTo("something")); - Map sObj2 = response.getHits().getAt(0).field("s_obj2").value(); + Map sObj2 = response.getHits().getAt(0).field("s_obj2").getValue(); List sObj2Arr2 = (List) sObj2.get("arr2"); assertThat(sObj2Arr2.size(), equalTo(2)); assertThat(sObj2Arr2.get(0).toString(), equalTo("arr_value1")); assertThat(sObj2Arr2.get(1).toString(), equalTo("arr_value2")); - sObj2Arr2 = response.getHits().getAt(0).field("s_obj2_arr2").values(); + sObj2Arr2 = response.getHits().getAt(0).field("s_obj2_arr2").getValues(); assertThat(sObj2Arr2.size(), equalTo(2)); assertThat(sObj2Arr2.get(0).toString(), equalTo("arr_value1")); assertThat(sObj2Arr2.get(1).toString(), equalTo("arr_value2")); - List sObj2Arr3 = response.getHits().getAt(0).field("s_arr3").values(); + List sObj2Arr3 = response.getHits().getAt(0).field("s_arr3").getValues(); assertThat(((Map) sObj2Arr3.get(0)).get("arr3_field1").toString(), equalTo("arr3_value1")); } @@ -487,7 +487,7 @@ public class SearchFieldsIT extends ESIntegTestCase { SearchHitField fieldObj = response.getHits().getAt(0).field("test_script_1"); assertThat(fieldObj, notNullValue()); - List fieldValues = fieldObj.values(); + List fieldValues = fieldObj.getValues(); assertThat(fieldValues, hasSize(1)); assertThat(fieldValues.get(0), nullValue()); } @@ -589,23 +589,23 @@ public class SearchFieldsIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - Set fields = new HashSet<>(searchResponse.getHits().getAt(0).fields().keySet()); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + Set fields = new HashSet<>(searchResponse.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(newHashSet("byte_field", "short_field", "integer_field", "long_field", "float_field", "double_field", "date_field", "boolean_field", "binary_field"))); SearchHit searchHit = searchResponse.getHits().getAt(0); - assertThat(searchHit.fields().get("byte_field").value().toString(), equalTo("1")); - assertThat(searchHit.fields().get("short_field").value().toString(), equalTo("2")); - assertThat(searchHit.fields().get("integer_field").value(), equalTo((Object) 3)); - assertThat(searchHit.fields().get("long_field").value(), equalTo((Object) 4L)); - assertThat(searchHit.fields().get("float_field").value(), equalTo((Object) 5.0f)); - assertThat(searchHit.fields().get("double_field").value(), equalTo((Object) 6.0d)); + assertThat(searchHit.getFields().get("byte_field").getValue().toString(), equalTo("1")); + assertThat(searchHit.getFields().get("short_field").getValue().toString(), equalTo("2")); + assertThat(searchHit.getFields().get("integer_field").getValue(), equalTo((Object) 3)); + assertThat(searchHit.getFields().get("long_field").getValue(), equalTo((Object) 4L)); + assertThat(searchHit.getFields().get("float_field").getValue(), equalTo((Object) 5.0f)); + assertThat(searchHit.getFields().get("double_field").getValue(), equalTo((Object) 6.0d)); String dateTime = Joda.forPattern("dateOptionalTime").printer().print(new DateTime(2012, 3, 22, 0, 0, DateTimeZone.UTC)); - assertThat(searchHit.fields().get("date_field").value(), equalTo((Object) dateTime)); - assertThat(searchHit.fields().get("boolean_field").value(), equalTo((Object) Boolean.TRUE)); - assertThat(searchHit.fields().get("binary_field").value(), equalTo(new BytesArray("testing text" .getBytes("UTF8")))); + assertThat(searchHit.getFields().get("date_field").getValue(), equalTo((Object) dateTime)); + assertThat(searchHit.getFields().get("boolean_field").getValue(), equalTo((Object) Boolean.TRUE)); + assertThat(searchHit.getFields().get("binary_field").getValue(), equalTo(new BytesArray("testing text" .getBytes("UTF8")))); } public void testSearchFieldsMetaData() throws Exception { @@ -620,7 +620,7 @@ public class SearchFieldsIT extends ESIntegTestCase { .addStoredField("field1").addStoredField("_routing") .get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); assertThat(searchResponse.getHits().getAt(0).field("field1"), nullValue()); assertThat(searchResponse.getHits().getAt(0).field("_routing").isMetadataField(), equalTo(true)); assertThat(searchResponse.getHits().getAt(0).field("_routing").getValue().toString(), equalTo("1")); @@ -697,14 +697,14 @@ public class SearchFieldsIT extends ESIntegTestCase { String field = "field1.field2.field3.field4"; SearchResponse searchResponse = client().prepareSearch("my-index").setTypes("my-type1").addStoredField(field).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); assertThat(searchResponse.getHits().getAt(0).field(field).isMetadataField(), equalTo(false)); assertThat(searchResponse.getHits().getAt(0).field(field).getValues().size(), equalTo(2)); assertThat(searchResponse.getHits().getAt(0).field(field).getValues().get(0).toString(), equalTo("value1")); assertThat(searchResponse.getHits().getAt(0).field(field).getValues().get(1).toString(), equalTo("value2")); searchResponse = client().prepareSearch("my-index").setTypes("my-type2").addStoredField(field).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); assertThat(searchResponse.getHits().getAt(0).field(field).isMetadataField(), equalTo(false)); assertThat(searchResponse.getHits().getAt(0).field(field).getValues().size(), equalTo(2)); assertThat(searchResponse.getHits().getAt(0).field(field).getValues().get(0).toString(), equalTo("value1")); @@ -721,7 +721,7 @@ public class SearchFieldsIT extends ESIntegTestCase { new SearchSourceBuilder().query(QueryBuilders.matchAllQuery()).fieldDataField("test_field")).get(); assertHitCount(searchResponse, 1); Map fields = searchResponse.getHits().getHits()[0].getFields(); - assertThat(fields.get("test_field").value(), equalTo("foobar")); + assertThat(fields.get("test_field").getValue(), equalTo("foobar")); } public void testFieldsPulledFromFieldData() throws Exception { @@ -810,23 +810,23 @@ public class SearchFieldsIT extends ESIntegTestCase { SearchResponse searchResponse = builder.execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - Set fields = new HashSet<>(searchResponse.getHits().getAt(0).fields().keySet()); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + Set fields = new HashSet<>(searchResponse.getHits().getAt(0).getFields().keySet()); assertThat(fields, equalTo(newHashSet("byte_field", "short_field", "integer_field", "long_field", "float_field", "double_field", "date_field", "boolean_field", "text_field", "keyword_field", "ip_field"))); - assertThat(searchResponse.getHits().getAt(0).fields().get("byte_field").value().toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(0).fields().get("short_field").value().toString(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(0).fields().get("integer_field").value(), equalTo((Object) 3L)); - assertThat(searchResponse.getHits().getAt(0).fields().get("long_field").value(), equalTo((Object) 4L)); - assertThat(searchResponse.getHits().getAt(0).fields().get("float_field").value(), equalTo((Object) 5.0)); - assertThat(searchResponse.getHits().getAt(0).fields().get("double_field").value(), equalTo((Object) 6.0d)); - assertThat(searchResponse.getHits().getAt(0).fields().get("date_field").value(), equalTo(date)); - assertThat(searchResponse.getHits().getAt(0).fields().get("boolean_field").value(), equalTo((Object) true)); - assertThat(searchResponse.getHits().getAt(0).fields().get("text_field").value(), equalTo("foo")); - assertThat(searchResponse.getHits().getAt(0).fields().get("keyword_field").value(), equalTo("foo")); - assertThat(searchResponse.getHits().getAt(0).fields().get("ip_field").value(), equalTo("::1")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("byte_field").getValue().toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("short_field").getValue().toString(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("integer_field").getValue(), equalTo((Object) 3L)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("long_field").getValue(), equalTo((Object) 4L)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("float_field").getValue(), equalTo((Object) 5.0)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("double_field").getValue(), equalTo((Object) 6.0d)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("date_field").getValue(), equalTo(date)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("boolean_field").getValue(), equalTo((Object) true)); + assertThat(searchResponse.getHits().getAt(0).getFields().get("text_field").getValue(), equalTo("foo")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("keyword_field").getValue(), equalTo("foo")); + assertThat(searchResponse.getHits().getAt(0).getFields().get("ip_field").getValue(), equalTo("::1")); } public void testScriptFields() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java index 13a2c3a1c3a..d119d9bacf1 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java @@ -214,7 +214,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { assertThat(sh.getTotalHits(), equalTo((long) (numDummyDocs + 2))); assertThat(sh.getAt(0).getId(), anyOf(equalTo("1"), equalTo("2"))); assertThat(sh.getAt(1).getId(), anyOf(equalTo("1"), equalTo("2"))); - assertThat(sh.getAt(1).score(), equalTo(sh.getAt(0).score())); + assertThat(sh.getAt(1).getScore(), equalTo(sh.getAt(0).getScore())); for (int i = 0; i < numDummyDocs; i++) { assertThat(sh.getAt(i + 2).getId(), equalTo(Integer.toString(i + 3))); } @@ -233,7 +233,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { assertThat(sh.getTotalHits(), equalTo((long) (numDummyDocs + 2))); assertThat(sh.getAt(0).getId(), anyOf(equalTo("1"), equalTo("2"))); assertThat(sh.getAt(1).getId(), anyOf(equalTo("1"), equalTo("2"))); - assertThat(sh.getAt(1).score(), equalTo(sh.getAt(0).score())); + assertThat(sh.getAt(1).getScore(), equalTo(sh.getAt(0).getScore())); for (int i = 0; i < numDummyDocs; i++) { assertThat(sh.getAt(i + 2).getId(), equalTo(Integer.toString(i + 3))); } @@ -249,7 +249,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { assertThat(sh.getTotalHits(), equalTo((long) (numDummyDocs + 2))); assertThat(sh.getAt(0).getId(), anyOf(equalTo("1"), equalTo("2"))); assertThat(sh.getAt(1).getId(), anyOf(equalTo("1"), equalTo("2"))); - assertThat(sh.getAt(1).score(), equalTo(sh.getAt(0).score())); + assertThat(sh.getAt(1).getScore(), equalTo(sh.getAt(0).getScore())); } public void testBoostModeSettingWorks() throws Exception { @@ -344,7 +344,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { SearchHits sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(1.0, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(1.0, 1.e-5)); // this is equivalent to new GeoPoint(20, 11); just flipped so scores must be same float[] coords = { 11, 20 }; response = client().search( @@ -356,7 +356,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(1.0f, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(1.0f, 1.e-5)); } public void testCombineModes() throws Exception { @@ -379,7 +379,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { SearchHits sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(1.0, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(1.0, 1.e-5)); response = client().search( searchRequest().searchType(SearchType.QUERY_THEN_FETCH).source( @@ -390,7 +390,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(0.5, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(0.5, 1.e-5)); response = client().search( searchRequest().searchType(SearchType.QUERY_THEN_FETCH).source( @@ -401,8 +401,8 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(2.0 + 0.5, 1.e-5)); - logger.info("--> Hit[0] {} Explanation:\n {}", sr.getHits().getAt(0).id(), sr.getHits().getAt(0).explanation()); + assertThat((double) sh.getAt(0).getScore(), closeTo(2.0 + 0.5, 1.e-5)); + logger.info("--> Hit[0] {} Explanation:\n {}", sr.getHits().getAt(0).getId(), sr.getHits().getAt(0).getExplanation()); response = client().search( searchRequest().searchType(SearchType.QUERY_THEN_FETCH).source( @@ -413,7 +413,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo((2.0 + 0.5) / 2, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo((2.0 + 0.5) / 2, 1.e-5)); response = client().search( searchRequest().searchType(SearchType.QUERY_THEN_FETCH).source( @@ -424,7 +424,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(0.5, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(0.5, 1.e-5)); response = client().search( searchRequest().searchType(SearchType.QUERY_THEN_FETCH).source( @@ -435,7 +435,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { sh = sr.getHits(); assertThat(sh.getTotalHits(), equalTo((long) (1))); assertThat(sh.getAt(0).getId(), equalTo("1")); - assertThat((double) sh.getAt(0).score(), closeTo(2.0, 1.e-5)); + assertThat((double) sh.getAt(0).getScore(), closeTo(2.0, 1.e-5)); } @@ -543,9 +543,9 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { assertNoFailures(sr); SearchHits sh = sr.getHits(); - assertThat(sh.hits().length, equalTo(4)); + assertThat(sh.getHits().length, equalTo(4)); double[] scores = new double[4]; - for (int i = 0; i < sh.hits().length; i++) { + for (int i = 0; i < sh.getHits().length; i++) { scores[Integer.parseInt(sh.getAt(i).getId()) - 1] = sh.getAt(i).getScore(); } assertThat(scores[0], lessThan(scores[1])); @@ -594,9 +594,9 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { SearchResponse sr = response.actionGet(); assertNoFailures(sr); SearchHits sh = sr.getHits(); - assertThat(sh.hits().length, equalTo(3)); + assertThat(sh.getHits().length, equalTo(3)); double[] scores = new double[4]; - for (int i = 0; i < sh.hits().length; i++) { + for (int i = 0; i < sh.getHits().length; i++) { scores[Integer.parseInt(sh.getAt(i).getId()) - 1] = sh.getAt(i).getScore(); } assertThat(scores[1], lessThan(scores[0])); @@ -648,7 +648,7 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { SearchResponse sr = response.actionGet(); assertNoFailures(sr); SearchHits sh = sr.getHits(); - assertThat(sh.hits().length, equalTo(numDocs)); + assertThat(sh.getHits().length, equalTo(numDocs)); double[] scores = new double[numDocs]; for (int i = 0; i < numDocs; i++) { scores[Integer.parseInt(sh.getAt(i).getId())] = sh.getAt(i).getScore(); diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java index 9b732bdc00d..b4e15e06849 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java @@ -87,10 +87,10 @@ public class ExplainableScriptIT extends ESIntegTestCase { int idCounter = 19; for (SearchHit hit : hits.getHits()) { assertThat(hit.getId(), equalTo(Integer.toString(idCounter))); - assertThat(hit.explanation().toString(), + assertThat(hit.getExplanation().toString(), containsString(Double.toString(idCounter) + " = This script returned " + Double.toString(idCounter))); - assertThat(hit.explanation().toString(), containsString("freq=1.0 = termFreq=1.0")); - assertThat(hit.explanation().getDetails().length, equalTo(2)); + assertThat(hit.getExplanation().toString(), containsString("freq=1.0 = termFreq=1.0")); + assertThat(hit.getExplanation().getDetails().length, equalTo(2)); idCounter--; } } diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreFieldValueIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreFieldValueIT.java index b850a028bcf..2907a49288a 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreFieldValueIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreFieldValueIT.java @@ -110,7 +110,7 @@ public class FunctionScoreFieldValueIT extends ESIntegTestCase { .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("notmapped").modifier(FieldValueFactorFunction.Modifier.RECIPROCAL).missing(100))) .get(); - assertEquals(response.getHits().getAt(0).score(), response.getHits().getAt(2).score(), 0); + assertEquals(response.getHits().getAt(0).getScore(), response.getHits().getAt(2).getScore(), 0); // n divided by 0 is infinity, which should provoke an exception. diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreIT.java index e9cb9d72a8b..dd608a38053 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScoreIT.java @@ -110,7 +110,7 @@ public class FunctionScoreIT extends ESIntegTestCase { ) ).actionGet(); assertSearchResponse(response); - assertThat(response.getHits().getAt(0).score(), equalTo(1.0f)); + assertThat(response.getHits().getAt(0).getScore(), equalTo(1.0f)); } public void testScriptScoresWithAgg() throws IOException { @@ -128,7 +128,7 @@ public class FunctionScoreIT extends ESIntegTestCase { ) ).actionGet(); assertSearchResponse(response); - assertThat(response.getHits().getAt(0).score(), equalTo(1.0f)); + assertThat(response.getHits().getAt(0).getScore(), equalTo(1.0f)); assertThat(((Terms) response.getAggregations().asMap().get("score_agg")).getBuckets().get(0).getKeyAsString(), equalTo("1.0")); assertThat(((Terms) response.getAggregations().asMap().get("score_agg")).getBuckets().get(0).getDocCount(), is(1L)); } @@ -202,9 +202,9 @@ public class FunctionScoreIT extends ESIntegTestCase { protected void assertMinScoreSearchResponses(int numDocs, SearchResponse searchResponse, int numMatchingDocs) { assertSearchResponse(searchResponse); - assertThat((int) searchResponse.getHits().totalHits(), is(numMatchingDocs)); + assertThat((int) searchResponse.getHits().getTotalHits(), is(numMatchingDocs)); int pos = 0; - for (int hitId = numDocs - 1; (numDocs - hitId) < searchResponse.getHits().totalHits(); hitId--) { + for (int hitId = numDocs - 1; (numDocs - hitId) < searchResponse.getHits().getTotalHits(); hitId--) { assertThat(searchResponse.getHits().getAt(pos).getId(), equalTo(Integer.toString(hitId))); pos++; } @@ -221,7 +221,7 @@ public class FunctionScoreIT extends ESIntegTestCase { searchSource().explain(true).query( termQuery("text", "text")))).get(); assertSearchResponse(termQuery); - assertThat(termQuery.getHits().totalHits(), equalTo(1L)); + assertThat(termQuery.getHits().getTotalHits(), equalTo(1L)); float termQueryScore = termQuery.getHits().getAt(0).getScore(); for (CombineFunction combineFunction : CombineFunction.values()) { @@ -235,7 +235,7 @@ public class FunctionScoreIT extends ESIntegTestCase { searchSource().explain(true).query( functionScoreQuery(termQuery("text", "text")).boostMode(boostMode).setMinScore(0.1f)))).get(); assertSearchResponse(response); - assertThat(response.getHits().totalHits(), equalTo(1L)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); assertThat(response.getHits().getAt(0).getScore(), equalTo(expectedScore)); response = client().search( @@ -244,7 +244,7 @@ public class FunctionScoreIT extends ESIntegTestCase { functionScoreQuery(termQuery("text", "text")).boostMode(boostMode).setMinScore(2f)))).get(); assertSearchResponse(response); - assertThat(response.getHits().totalHits(), equalTo(0L)); + assertThat(response.getHits().getTotalHits(), equalTo(0L)); } } diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java index 21622d0e4ae..f8487cad059 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/FunctionScorePluginIT.java @@ -92,7 +92,7 @@ public class FunctionScorePluginIT extends ESIntegTestCase { ElasticsearchAssertions.assertNoFailures(sr); SearchHits sh = sr.getHits(); - assertThat(sh.hits().length, equalTo(2)); + assertThat(sh.getHits().length, equalTo(2)); assertThat(sh.getAt(0).getId(), equalTo("1")); assertThat(sh.getAt(1).getId(), equalTo("2")); diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java index 242847587f6..fc9dfb88449 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java @@ -92,13 +92,13 @@ public class QueryRescorerIT extends ESIntegTestCase { assertSearchResponse(searchResponse); assertFirstHit(searchResponse, hasScore(100.f)); int numDocsWith100AsAScore = 0; - for (int i = 0; i < searchResponse.getHits().hits().length; i++) { - float score = searchResponse.getHits().hits()[i].getScore(); + for (int i = 0; i < searchResponse.getHits().getHits().length; i++) { + float score = searchResponse.getHits().getHits()[i].getScore(); if (score == 100f) { numDocsWith100AsAScore += 1; } } - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); // we cannot assert that they are equal since some shards might not have docs at all assertThat(numDocsWith100AsAScore, lessThanOrEqualTo(numShards)); } @@ -123,8 +123,8 @@ public class QueryRescorerIT extends ESIntegTestCase { queryRescorer(QueryBuilders.matchPhraseQuery("field1", "quick brown").slop(2).boost(4.0f)) .setRescoreQueryWeight(2), 5).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("3")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("2")); @@ -145,7 +145,7 @@ public class QueryRescorerIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 3); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("1")); assertSecondHit(searchResponse, hasId("2")); assertThirdHit(searchResponse, hasId("3")); @@ -190,7 +190,7 @@ public class QueryRescorerIT extends ESIntegTestCase { .setRescorer(queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3)) .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f), 20).execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(5)); + assertThat(searchResponse.getHits().getHits().length, equalTo(5)); assertHitCount(searchResponse, 9); assertFirstHit(searchResponse, hasId("2")); assertSecondHit(searchResponse, hasId("6")); @@ -205,9 +205,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setRescorer(queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3)) .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f), 20).execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(5)); + assertThat(searchResponse.getHits().getHits().length, equalTo(5)); assertHitCount(searchResponse, 9); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("2")); assertSecondHit(searchResponse, hasId("6")); assertThirdHit(searchResponse, hasId("3")); @@ -222,9 +222,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setRescorer(queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3)) .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f), 20).execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(5)); + assertThat(searchResponse.getHits().getHits().length, equalTo(5)); assertHitCount(searchResponse, 9); - assertThat(searchResponse.getHits().maxScore(), greaterThan(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), greaterThan(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("3")); } @@ -256,9 +256,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setQuery(QueryBuilders.matchQuery("field1", "massachusetts")) .setFrom(0) .setSize(5).execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("3")); assertSecondHit(searchResponse, hasId("6")); assertThirdHit(searchResponse, hasId("1")); @@ -273,9 +273,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setRescorer(queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3)) .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f), 2).execute().actionGet(); // Only top 2 hits were re-ordered: - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("6")); assertSecondHit(searchResponse, hasId("3")); assertThirdHit(searchResponse, hasId("1")); @@ -291,9 +291,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f), 3).execute().actionGet(); // Only top 3 hits were re-ordered: - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("6")); assertSecondHit(searchResponse, hasId("1")); assertThirdHit(searchResponse, hasId("3")); @@ -328,9 +328,9 @@ public class QueryRescorerIT extends ESIntegTestCase { .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(Operator.OR)) .setFrom(0) .setSize(5).execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("3")); assertSecondHit(searchResponse, hasId("6")); assertThirdHit(searchResponse, hasId("1")); @@ -346,7 +346,7 @@ public class QueryRescorerIT extends ESIntegTestCase { .setQueryWeight(1.0f).setRescoreQueryWeight(-1f), 3).execute().actionGet(); // 6 and 1 got worse, and then the hit (2) outside the rescore window were sorted ahead: - assertThat(searchResponse.getHits().maxScore(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getMaxScore(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertFirstHit(searchResponse, hasId("3")); assertSecondHit(searchResponse, hasId("2")); assertThirdHit(searchResponse, hasId("6")); @@ -363,7 +363,7 @@ public class QueryRescorerIT extends ESIntegTestCase { public int compare(SearchHit hit1, SearchHit hit2) { int cmp = Float.compare(hit2.getScore(), hit1.getScore()); if (cmp == 0) { - return hit1.id().compareTo(hit2.id()); + return hit1.getId().compareTo(hit2.getId()); } else { return cmp; } @@ -402,7 +402,7 @@ public class QueryRescorerIT extends ESIntegTestCase { SearchHit[] hits = leftHits.getHits(); SearchHit[] otherHits = rightHits.getHits(); if (!hits[0].getId().equals(otherHits[0].getId())) { - assertThat(((String) otherHits[0].sourceAsMap().get("field1")).contains(query), equalTo(true)); + assertThat(((String) otherHits[0].getSourceAsMap().get("field1")).contains(query), equalTo(true)); } else { Arrays.sort(hits, searchHitsComparator); Arrays.sort(otherHits, searchHitsComparator); @@ -506,16 +506,18 @@ public class QueryRescorerIT extends ESIntegTestCase { assertThirdHit(searchResponse, hasId("3")); for (int i = 0; i < 3; i++) { - assertThat(searchResponse.getHits().getAt(i).explanation(), notNullValue()); - assertThat(searchResponse.getHits().getAt(i).explanation().isMatch(), equalTo(true)); - assertThat(searchResponse.getHits().getAt(i).explanation().getDetails().length, equalTo(2)); - assertThat(searchResponse.getHits().getAt(i).explanation().getDetails()[0].isMatch(), equalTo(true)); + assertThat(searchResponse.getHits().getAt(i).getExplanation(), notNullValue()); + assertThat(searchResponse.getHits().getAt(i).getExplanation().isMatch(), equalTo(true)); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDetails().length, equalTo(2)); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDetails()[0].isMatch(), equalTo(true)); if (i == 2) { - assertThat(searchResponse.getHits().getAt(i).explanation().getDetails()[1].getValue(), equalTo(0.5f)); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDetails()[1].getValue(), equalTo(0.5f)); } else { - assertThat(searchResponse.getHits().getAt(i).explanation().getDescription(), equalTo("sum of:")); - assertThat(searchResponse.getHits().getAt(i).explanation().getDetails()[0].getDetails()[1].getValue(), equalTo(0.5f)); - assertThat(searchResponse.getHits().getAt(i).explanation().getDetails()[1].getDetails()[1].getValue(), equalTo(0.4f)); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDescription(), equalTo("sum of:")); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDetails()[0].getDetails()[1].getValue(), + equalTo(0.5f)); + assertThat(searchResponse.getHits().getAt(i).getExplanation().getDetails()[1].getDetails()[1].getValue(), + equalTo(0.4f)); } } } @@ -542,7 +544,7 @@ public class QueryRescorerIT extends ESIntegTestCase { assertThirdHit(searchResponse, hasId("3")); for (int j = 0; j < 3; j++) { - assertThat(searchResponse.getHits().getAt(j).explanation().getDescription(), equalTo(descriptionModes[innerMode])); + assertThat(searchResponse.getHits().getAt(j).getExplanation().getDescription(), equalTo(descriptionModes[innerMode])); } for (int outerMode = 0; outerMode < scoreModes.length; outerMode++) { @@ -565,7 +567,7 @@ public class QueryRescorerIT extends ESIntegTestCase { assertThirdHit(searchResponse, hasId("3")); for (int j = 0; j < 3; j++) { - Explanation explanation = searchResponse.getHits().getAt(j).explanation(); + Explanation explanation = searchResponse.getHits().getAt(j).getExplanation(); assertThat(explanation.getDescription(), equalTo(descriptionModes[outerMode])); assertThat(explanation.getDetails()[0].getDetails()[0].getDescription(), equalTo(descriptionModes[innerMode])); } @@ -606,7 +608,7 @@ public class QueryRescorerIT extends ESIntegTestCase { assertHitCount(rescored, 4); - assertThat(rescored.getHits().maxScore(), equalTo(rescored.getHits().getHits()[0].score())); + assertThat(rescored.getHits().getMaxScore(), equalTo(rescored.getHits().getHits()[0].getScore())); if ("total".equals(scoreMode) || "".equals(scoreMode)) { assertFirstHit(rescored, hasId(String.valueOf(i + 1))); assertSecondHit(rescored, hasId(String.valueOf(i))); @@ -684,7 +686,7 @@ public class QueryRescorerIT extends ESIntegTestCase { .boostMode(CombineFunction.REPLACE)).setScoreMode(QueryRescoreMode.Total); request.clearRescorers().addRescorer(ninetyIsGood, numDocs).addRescorer(oneToo, 10); response = request.setSize(2).get(); - assertThat(response.getHits().maxScore(), equalTo(response.getHits().getHits()[0].score())); + assertThat(response.getHits().getMaxScore(), equalTo(response.getHits().getHits()[0].getScore())); assertFirstHit(response, hasId("91")); assertFirstHit(response, hasScore(2001.0f)); assertSecondHit(response, hasScore(1001.0f)); // Not sure which one it is but it is ninety something @@ -735,6 +737,6 @@ public class QueryRescorerIT extends ESIntegTestCase { request.setSize(4); request.addRescorer(RescoreBuilder.queryRescorer(QueryBuilders.matchAllQuery()), 50); - assertEquals(4, request.get().getHits().hits().length); + assertEquals(4, request.get().getHits().getHits().length); } } diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/RandomScoreFunctionIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/RandomScoreFunctionIT.java index a6f7eb760d2..35e3754d115 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/RandomScoreFunctionIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/RandomScoreFunctionIT.java @@ -134,8 +134,8 @@ public class RandomScoreFunctionIT extends ESIntegTestCase { } else { assertThat(hits.length, equalTo(searchResponse.getHits().getHits().length)); for (int j = 0; j < hitCount; j++) { - assertThat("" + j, currentHits[j].score(), equalTo(hits[j].score())); - assertThat("" + j, currentHits[j].id(), equalTo(hits[j].id())); + assertThat("" + j, currentHits[j].getScore(), equalTo(hits[j].getScore())); + assertThat("" + j, currentHits[j].getId(), equalTo(hits[j].getId())); } } @@ -263,9 +263,9 @@ public class RandomScoreFunctionIT extends ESIntegTestCase { .setExplain(true) .get(); assertNoFailures(resp); - assertEquals(1, resp.getHits().totalHits()); + assertEquals(1, resp.getHits().getTotalHits()); SearchHit firstHit = resp.getHits().getAt(0); - assertThat(firstHit.explanation().toString(), containsString("" + seed)); + assertThat(firstHit.getExplanation().toString(), containsString("" + seed)); } public void testNoDocs() throws Exception { @@ -276,7 +276,7 @@ public class RandomScoreFunctionIT extends ESIntegTestCase { .setQuery(functionScoreQuery(matchAllQuery(), randomFunction(1234))) .get(); assertNoFailures(resp); - assertEquals(0, resp.getHits().totalHits()); + assertEquals(0, resp.getHits().getTotalHits()); } public void testScoreRange() throws Exception { @@ -300,7 +300,7 @@ public class RandomScoreFunctionIT extends ESIntegTestCase { assertNoFailures(searchResponse); for (SearchHit hit : searchResponse.getHits().getHits()) { - assertThat(hit.score(), allOf(greaterThanOrEqualTo(0.0f), lessThanOrEqualTo(1.0f))); + assertThat(hit.getScore(), allOf(greaterThanOrEqualTo(0.0f), lessThanOrEqualTo(1.0f))); } } } @@ -351,7 +351,7 @@ public class RandomScoreFunctionIT extends ESIntegTestCase { .setQuery(functionScoreQuery(matchAllQuery(), new RandomScoreFunctionBuilder())) .execute().actionGet(); - matrix[Integer.valueOf(searchResponse.getHits().getAt(0).id())]++; + matrix[Integer.valueOf(searchResponse.getHits().getAt(0).getId())]++; } int filled = 0; diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java b/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java index 646cc310fc8..027781a9b2b 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java @@ -106,18 +106,18 @@ public class GeoBoundingBoxIT extends ESIntegTestCase { .setQuery(geoBoundingBoxQuery("location").setCorners(40.73, -74.1, 40.717, -73.99)) .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id(), anyOf(equalTo("1"), equalTo("3"), equalTo("5"))); + assertThat(hit.getId(), anyOf(equalTo("1"), equalTo("3"), equalTo("5"))); } searchResponse = client().prepareSearch() // from NY .setQuery(geoBoundingBoxQuery("location").setCorners(40.73, -74.1, 40.717, -73.99).type("indexed")) .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id(), anyOf(equalTo("1"), equalTo("3"), equalTo("5"))); + assertThat(hit.getId(), anyOf(equalTo("1"), equalTo("3"), equalTo("5"))); } } @@ -151,26 +151,26 @@ public class GeoBoundingBoxIT extends ESIntegTestCase { boolQuery().must(termQuery("userid", 880)).filter( geoBoundingBoxQuery("location").setCorners(74.579421999999994, 143.5, -66.668903999999998, 113.96875)) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( boolQuery().must(termQuery("userid", 880)).filter( geoBoundingBoxQuery("location").setCorners(74.579421999999994, 143.5, -66.668903999999998, 113.96875).type("indexed")) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( boolQuery().must(termQuery("userid", 534)).filter( geoBoundingBoxQuery("location").setCorners(74.579421999999994, 143.5, -66.668903999999998, 113.96875)) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( boolQuery().must(termQuery("userid", 534)).filter( geoBoundingBoxQuery("location").setCorners(74.579421999999994, 143.5, -66.668903999999998, 113.96875).type("indexed")) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testCompleteLonRange() throws Exception { @@ -202,43 +202,43 @@ public class GeoBoundingBoxIT extends ESIntegTestCase { .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(50, -180, -50, 180) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(50, -180, -50, 180).type("indexed") ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(90, -180, -90, 180) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(90, -180, -90, 180).type("indexed") ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(50, 0, -50, 360) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(50, 0, -50, 360).type("indexed") ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(90, 0, -90, 360) ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); searchResponse = client().prepareSearch() .setQuery( geoBoundingBoxQuery("location").setValidationMethod(GeoValidationMethod.COERCE).setCorners(90, 0, -90, 360).type("indexed") ).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); } } diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java b/core/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java index 1d864c14dec..e3678683a09 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java @@ -111,9 +111,9 @@ public class GeoPolygonIT extends ESIntegTestCase { .setQuery(boolQuery().must(geoPolygonQuery("location", points))) .execute().actionGet(); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id(), anyOf(equalTo("1"), equalTo("3"), equalTo("4"), equalTo("5"))); + assertThat(hit.getId(), anyOf(equalTo("1"), equalTo("3"), equalTo("4"), equalTo("5"))); } } @@ -126,9 +126,9 @@ public class GeoPolygonIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test") // from NY .setQuery(boolQuery().must(geoPolygonQuery("location", points))).execute().actionGet(); assertHitCount(searchResponse, 4); - assertThat(searchResponse.getHits().hits().length, equalTo(4)); + assertThat(searchResponse.getHits().getHits().length, equalTo(4)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id(), anyOf(equalTo("1"), equalTo("3"), equalTo("4"), equalTo("5"))); + assertThat(hit.getId(), anyOf(equalTo("1"), equalTo("3"), equalTo("4"), equalTo("5"))); } } } diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index 7e8aa1ad9d0..f0efe523bc6 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -110,8 +110,8 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { assertSearchResponse(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); searchResponse = client().prepareSearch("test").setTypes("type1") .setQuery(geoShapeQuery("location", shape)) @@ -119,8 +119,8 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { assertSearchResponse(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); } public void testEdgeCases() throws Exception { @@ -156,8 +156,8 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { assertSearchResponse(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("blakely")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("blakely")); } public void testIndexedShapeReference() throws Exception { @@ -189,8 +189,8 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { assertSearchResponse(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); searchResponse = client().prepareSearch("test") .setQuery(geoShapeQuery("location", "Big_Rectangle", "shape_type")) @@ -198,8 +198,8 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { assertSearchResponse(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); } public void testIndexedShapeReferenceSourceDisabled() throws Exception { @@ -369,7 +369,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .setPostFilter(filter).get(); assertSearchResponse(response); - assertThat(response.getHits().totalHits(), greaterThan(0L)); + assertThat(response.getHits().getTotalHits(), greaterThan(0L)); } public void testShapeFilterWithDefinedGeoCollection() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java b/core/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java index ab42a09a6fd..9ee41e9e37d 100644 --- a/core/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java +++ b/core/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java @@ -127,19 +127,19 @@ public class MoreLikeThisIT extends ESIntegTestCase { response = client().prepareSearch("beta").setQuery( new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); - assertThat(response.getHits().getAt(0).id(), equalTo("3")); + assertThat(response.getHits().getAt(0).getId(), equalTo("3")); logger.info("Running moreLikeThis on release shard"); response = client().prepareSearch("release").setQuery( new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); - assertThat(response.getHits().getAt(0).id(), equalTo("2")); + assertThat(response.getHits().getAt(0).getId(), equalTo("2")); logger.info("Running moreLikeThis on alias with node client"); response = internalCluster().coordOnlyNodeClient().prepareSearch("beta").setQuery( new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); - assertThat(response.getHits().getAt(0).id(), equalTo("3")); + assertThat(response.getHits().getAt(0).getId(), equalTo("3")); } // Issue #14944 @@ -165,7 +165,7 @@ public class MoreLikeThisIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch().setQuery( new MoreLikeThisQueryBuilder(null, new Item[] {new Item(aliasName, typeName, "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 2L); - assertThat(response.getHits().getAt(0).id(), equalTo("3")); + assertThat(response.getHits().getAt(0).getId(), equalTo("3")); } public void testMoreLikeThisIssue2197() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java b/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java index a7a59db4d4d..718893b42db 100644 --- a/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java +++ b/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java @@ -63,9 +63,9 @@ public class SimpleNestedIT extends ESIntegTestCase { // check on no data, see it works SearchResponse searchResponse = client().prepareSearch("test").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); searchResponse = client().prepareSearch("test").setQuery(termQuery("n_field1", "n_value1_1")).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() .field("field1", "value1") @@ -92,22 +92,22 @@ public class SimpleNestedIT extends ESIntegTestCase { assertDocumentCount("test", 3); searchResponse = client().prepareSearch("test").setQuery(termQuery("n_field1", "n_value1_1")).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); // search for something that matches the nested doc, and see that we don't find the nested doc searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(termQuery("n_field1", "n_value1_1")).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); // now, do a nested query searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1_1"), ScoreMode.Avg)).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1_1"), ScoreMode.Avg)).setSearchType(SearchType.DFS_QUERY_THEN_FETCH).get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); // add another doc, one that would match if it was not nested... @@ -132,19 +132,19 @@ public class SimpleNestedIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.n_field1", "n_value1_1")).must(termQuery("nested1.n_field2", "n_value2_1")), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); // filter searchResponse = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).mustNot(nestedQuery("nested1", boolQuery().must(termQuery("nested1.n_field1", "n_value1_1")).must(termQuery("nested1.n_field2", "n_value2_1")), ScoreMode.Avg))).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); // check with type prefix searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.n_field1", "n_value1_1")).must(termQuery("nested1.n_field2", "n_value2_1")), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); // check delete, so all is gone... DeleteResponse deleteResponse = client().prepareDelete("test", "type1", "2").execute().actionGet(); @@ -156,11 +156,11 @@ public class SimpleNestedIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1_1"), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setTypes("type1", "type2").setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1_1"), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testMultiNested() throws Exception { @@ -194,42 +194,42 @@ public class SimpleNestedIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", termQuery("nested1.field1", "1"), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "2"), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "1")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "2"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "1")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "3"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "1")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "4"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "1")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "5"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "4")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "5"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch("test").setQuery(nestedQuery("nested1", boolQuery().must(termQuery("nested1.field1", "4")).must(nestedQuery("nested1.nested2", termQuery("nested1.nested2.field2", "2"), ScoreMode.Avg)), ScoreMode.Avg)).execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(0L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(0L)); } // When IncludeNestedDocsQuery is wrapped in a FilteredQuery then a in-finite loop occurs b/c of a bug in IncludeNestedDocsQuery#advance() @@ -315,9 +315,9 @@ public class SimpleNestedIT extends ESIntegTestCase { .setExplain(true) .execute().actionGet(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); - Explanation explanation = searchResponse.getHits().hits()[0].explanation(); - assertThat(explanation.getValue(), equalTo(searchResponse.getHits().getHits()[0].score())); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); + Explanation explanation = searchResponse.getHits().getHits()[0].getExplanation(); + assertThat(explanation.getValue(), equalTo(searchResponse.getHits().getHits()[0].getScore())); assertThat(explanation.toString(), startsWith("0.36464313 = sum of:\n 0.36464313 = Score based on 2 child docs in range from 0 to 1")); } @@ -381,12 +381,12 @@ public class SimpleNestedIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 3); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("2")); - assertThat(searchResponse.getHits().hits()[0].sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("3")); - assertThat(searchResponse.getHits().hits()[1].sortValues()[0].toString(), equalTo("3")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[2].sortValues()[0].toString(), equalTo("4")); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("4")); searchResponse = client().prepareSearch("test") .setTypes("type1") @@ -395,12 +395,12 @@ public class SimpleNestedIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 3); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[0].sortValues()[0].toString(), equalTo("5")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("3")); - assertThat(searchResponse.getHits().hits()[1].sortValues()[0].toString(), equalTo("4")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("2")); - assertThat(searchResponse.getHits().hits()[2].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("5")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("4")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("2")); } public void testSimpleNestedSortingWithNestedFilterMissing() throws Exception { @@ -477,12 +477,12 @@ public class SimpleNestedIT extends ESIntegTestCase { SearchResponse searchResponse = searchRequestBuilder.get(); assertHitCount(searchResponse, 3); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("2")); - assertThat(searchResponse.getHits().hits()[0].sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[1].sortValues()[0].toString(), equalTo("4")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("3")); - assertThat(searchResponse.getHits().hits()[2].sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("4")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("10")); searchRequestBuilder = client().prepareSearch("test").setTypes("type1").setQuery(QueryBuilders.matchAllQuery()) .addSort(SortBuilders.fieldSort("nested1.field1").setNestedPath("nested1").setNestedFilter(termQuery("nested1.field2", true)).missing(10).order(SortOrder.DESC)); @@ -494,12 +494,12 @@ public class SimpleNestedIT extends ESIntegTestCase { searchResponse = searchRequestBuilder.get(); assertHitCount(searchResponse, 3); - assertThat(searchResponse.getHits().hits()[0].id(), equalTo("3")); - assertThat(searchResponse.getHits().hits()[0].sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().hits()[1].id(), equalTo("1")); - assertThat(searchResponse.getHits().hits()[1].sortValues()[0].toString(), equalTo("5")); - assertThat(searchResponse.getHits().hits()[2].id(), equalTo("2")); - assertThat(searchResponse.getHits().hits()[2].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("5")); + assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("2")); client().prepareClearScroll().addScrollId("_all").get(); } @@ -663,11 +663,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("-3")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("-3")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("-2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("-2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("-1")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("-1")); // With nested filter searchResponse = client().prepareSearch() @@ -682,11 +682,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); // Nested path should be automatically detected, expect same results as above search request searchResponse = client().prepareSearch() @@ -702,11 +702,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -721,11 +721,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -761,11 +761,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); // Sort mode: sum searchResponse = client().prepareSearch() @@ -781,11 +781,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("7")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("7")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("11")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("11")); searchResponse = client().prepareSearch() @@ -801,11 +801,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("11")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("11")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("7")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("7")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("2")); // Sort mode: sum with filter searchResponse = client().prepareSearch() @@ -822,11 +822,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); // Sort mode: avg searchResponse = client().prepareSearch() @@ -842,11 +842,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -861,11 +861,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("3")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("1")); // Sort mode: avg with filter searchResponse = client().prepareSearch() @@ -882,11 +882,11 @@ public class SimpleNestedIT extends ESIntegTestCase { assertHitCount(searchResponse, 3); assertThat(searchResponse.getHits().getHits().length, equalTo(3)); assertThat(searchResponse.getHits().getHits()[0].getId(), equalTo("1")); - assertThat(searchResponse.getHits().getHits()[0].sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getHits()[0].getSortValues()[0].toString(), equalTo("1")); assertThat(searchResponse.getHits().getHits()[1].getId(), equalTo("2")); - assertThat(searchResponse.getHits().getHits()[1].sortValues()[0].toString(), equalTo("2")); + assertThat(searchResponse.getHits().getHits()[1].getSortValues()[0].toString(), equalTo("2")); assertThat(searchResponse.getHits().getHits()[2].getId(), equalTo("3")); - assertThat(searchResponse.getHits().getHits()[2].sortValues()[0].toString(), equalTo("3")); + assertThat(searchResponse.getHits().getHits()[2].getSortValues()[0].toString(), equalTo("3")); } // Issue #9305 @@ -1017,12 +1017,12 @@ public class SimpleNestedIT extends ESIntegTestCase { .get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 2); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("fname1")); - assertThat(searchResponse.getHits().getAt(0).sortValues()[1].toString(), equalTo("fname1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("fname1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[1].toString(), equalTo("fname3")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("fname1")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[1].toString(), equalTo("fname1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("fname1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[1].toString(), equalTo("fname3")); } public void testCheckFixedBitSetCache() throws Exception { @@ -1072,7 +1072,7 @@ public class SimpleNestedIT extends ESIntegTestCase { .setQuery(nestedQuery("array1", termQuery("array1.field1", "value1"), ScoreMode.Avg)) .get(); assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), equalTo(5L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(5L)); } clusterStatsResponse = client().admin().cluster().prepareClusterStats().get(); assertThat(clusterStatsResponse.getIndicesStats().getSegments().getBitsetMemoryInBytes(), greaterThan(0L)); diff --git a/core/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java b/core/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java index 4b3b890a20d..9163ee572cf 100644 --- a/core/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java +++ b/core/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java @@ -91,9 +91,9 @@ public class SearchPreferenceIT extends ESIntegTestCase { final Client client = internalCluster().smartClient(); SearchResponse searchResponse = client.prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet(); - String firstNodeId = searchResponse.getHits().getAt(0).shard().getNodeId(); + String firstNodeId = searchResponse.getHits().getAt(0).getShard().getNodeId(); searchResponse = client.prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet(); - String secondNodeId = searchResponse.getHits().getAt(0).shard().getNodeId(); + String secondNodeId = searchResponse.getHits().getAt(0).getShard().getNodeId(); assertThat(firstNodeId, not(equalTo(secondNodeId))); } @@ -106,29 +106,29 @@ public class SearchPreferenceIT extends ESIntegTestCase { refresh(); SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_local").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_local").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_primary").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_primary").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica_first").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica_first").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("1234").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setPreference("1234").execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); } public void testReplicaPreference() throws Exception { @@ -146,13 +146,13 @@ public class SearchPreferenceIT extends ESIntegTestCase { } SearchResponse resp = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica_first").execute().actionGet(); - assertThat(resp.getHits().totalHits(), equalTo(1L)); + assertThat(resp.getHits().getTotalHits(), equalTo(1L)); client().admin().indices().prepareUpdateSettings("test").setSettings("{\"number_of_replicas\": 1}", XContentType.JSON).get(); ensureGreen("test"); resp = client().prepareSearch().setQuery(matchAllQuery()).setPreference("_replica").execute().actionGet(); - assertThat(resp.getHits().totalHits(), equalTo(1L)); + assertThat(resp.getHits().getTotalHits(), equalTo(1L)); } public void testThatSpecifyingNonExistingNodesReturnsUsefulError() throws Exception { @@ -221,7 +221,7 @@ public class SearchPreferenceIT extends ESIntegTestCase { for (int i = 0; i < 2; i++) { SearchResponse searchResponse = request.execute().actionGet(); assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); - hitNodes.add(searchResponse.getHits().getAt(0).shard().getNodeId()); + hitNodes.add(searchResponse.getHits().getAt(0).getShard().getNodeId()); } assertThat(hitNodes.size(), greaterThan(1)); } diff --git a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java index b6935f021d4..f20c575f0be 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java @@ -31,7 +31,6 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.profile.ProfileResult; import org.elasticsearch.search.profile.ProfileShardResult; -import org.elasticsearch.search.profile.query.CollectorResult; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; @@ -164,8 +163,8 @@ public class QueryProfilerIT extends ESIntegTestCase { } assertThat( - "Profile totalHits of [" + profileResponse.getHits().totalHits() + "] is not close to Vanilla totalHits [" - + vanillaResponse.getHits().totalHits() + "]", + "Profile totalHits of [" + profileResponse.getHits().getTotalHits() + "] is not close to Vanilla totalHits [" + + vanillaResponse.getHits().getTotalHits() + "]", vanillaResponse.getHits().getTotalHits(), equalTo(profileResponse.getHits().getTotalHits())); SearchHit[] vanillaHits = vanillaResponse.getHits().getHits(); diff --git a/core/src/test/java/org/elasticsearch/search/query/ExistsIT.java b/core/src/test/java/org/elasticsearch/search/query/ExistsIT.java index 978eff3674d..f597d425385 100644 --- a/core/src/test/java/org/elasticsearch/search/query/ExistsIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/ExistsIT.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.index.mapper.FieldNamesFieldMapper; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ESIntegTestCase; @@ -128,7 +127,7 @@ public class ExistsIT extends ESIntegTestCase { SearchResponse resp = client().prepareSearch("idx").setQuery(QueryBuilders.existsQuery(fieldName)).execute().actionGet(); assertSearchResponse(resp); try { - assertEquals(String.format(Locale.ROOT, "exists(%s, %d) mapping: %s response: %s", fieldName, count, mapping.string(), resp), count, resp.getHits().totalHits()); + assertEquals(String.format(Locale.ROOT, "exists(%s, %d) mapping: %s response: %s", fieldName, count, mapping.string(), resp), count, resp.getHits().getTotalHits()); } catch (AssertionError e) { for (SearchHit searchHit : allDocs.getHits()) { final String index = searchHit.getIndex(); diff --git a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java index b5067dbf74a..2002e7ce508 100644 --- a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java @@ -184,19 +184,19 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .setQuery(randomizeType(multiMatchQuery("marvel hero captain america", "full_name", "first_name", "last_name", "category") .operator(Operator.OR))).get(); Set topNIds = Sets.newHashSet("theone", "theother"); - for (int i = 0; i < searchResponse.getHits().hits().length; i++) { + for (int i = 0; i < searchResponse.getHits().getHits().length; i++) { topNIds.remove(searchResponse.getHits().getAt(i).getId()); // very likely that we hit a random doc that has the same score so orders are random since // the doc id is the tie-breaker } assertThat(topNIds, empty()); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("marvel hero captain america", "full_name", "first_name", "last_name", "category") .operator(Operator.OR).useDisMax(false).type(type))).get(); assertFirstHit(searchResponse, anyOf(hasId("theone"), hasId("theother"))); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("marvel hero", "full_name", "first_name", "last_name", "category") @@ -281,10 +281,10 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertThat("field: " + field + " query: " + builder.toString(), multiMatchResp.getHits().getTotalHits(), equalTo(matchResp.getHits().getTotalHits())); SearchHits hits = multiMatchResp.getHits(); if (field.startsWith("missing")) { - assertEquals(0, hits.hits().length); + assertEquals(0, hits.getHits().length); } - for (int j = 0; j < hits.hits().length; j++) { - assertThat(hits.getHits()[j].score(), equalTo(matchResp.getHits().getHits()[j].score())); + for (int j = 0; j < hits.getHits().length; j++) { + assertThat(hits.getHits()[j].getScore(), equalTo(matchResp.getHits().getHits()[j].getScore())); assertThat(hits.getHits()[j].getId(), equalTo(matchResp.getHits().getHits()[j].getId())); } } @@ -293,27 +293,27 @@ public class MultiMatchQueryIT extends ESIntegTestCase { public void testCutoffFreq() throws ExecutionException, InterruptedException { final long numDocs = client().prepareSearch("test").setSize(0) - .setQuery(matchAllQuery()).get().getHits().totalHits(); + .setQuery(matchAllQuery()).get().getHits().getTotalHits(); MatchQuery.Type type = randomBoolean() ? MatchQueryBuilder.DEFAULT_TYPE : MatchQuery.Type.BOOLEAN; Float cutoffFrequency = randomBoolean() ? Math.min(1, numDocs * 1.f / between(10, 20)) : 1.f / between(10, 20); SearchResponse searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("marvel hero captain america", "full_name", "first_name", "last_name", "category") .operator(Operator.OR).cutoffFrequency(cutoffFrequency))).get(); Set topNIds = Sets.newHashSet("theone", "theother"); - for (int i = 0; i < searchResponse.getHits().hits().length; i++) { + for (int i = 0; i < searchResponse.getHits().getHits().length; i++) { topNIds.remove(searchResponse.getHits().getAt(i).getId()); // very likely that we hit a random doc that has the same score so orders are random since // the doc id is the tie-breaker } assertThat(topNIds, empty()); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThanOrEqualTo(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThanOrEqualTo(searchResponse.getHits().getHits()[1].getScore())); cutoffFrequency = randomBoolean() ? Math.min(1, numDocs * 1.f / between(10, 20)) : 1.f / between(10, 20); searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("marvel hero captain america", "full_name", "first_name", "last_name", "category") .operator(Operator.OR).useDisMax(false).cutoffFrequency(cutoffFrequency).type(type))).get(); assertFirstHit(searchResponse, anyOf(hasId("theone"), hasId("theother"))); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); long size = searchResponse.getHits().getTotalHits(); searchResponse = client().prepareSearch("test") @@ -353,7 +353,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { public void testEquivalence() { final int numDocs = (int) client().prepareSearch("test").setSize(0) - .setQuery(matchAllQuery()).get().getHits().totalHits(); + .setQuery(matchAllQuery()).get().getHits().getTotalHits(); int numIters = scaledRandomIntBetween(5, 10); for (int i = 0; i < numIters; i++) { { @@ -455,7 +455,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .operator(Operator.OR))).get(); assertFirstHit(searchResponse, hasId("theother")); assertSecondHit(searchResponse, hasId("theone")); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("marvel hero", "full_name", "first_name", "last_name", "category") @@ -544,7 +544,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .analyzer("category") .operator(Operator.OR))).get(); assertFirstHit(searchResponse, anyOf(hasId("theother"), hasId("theone"))); - long numResults = searchResponse.getHits().totalHits(); + long numResults = searchResponse.getHits().getTotalHits(); searchResponse = client().prepareSearch("test") .setQuery(randomizeType(multiMatchQuery("captain america marvel hero", "first_name", "last_name", "category") @@ -585,7 +585,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertHitCount(searchResponse, 2L); assertFirstHit(searchResponse, hasId("ultimate1")); // has ultimate in the last_name and that is boosted assertSecondHit(searchResponse, hasId("ultimate2")); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); // since we try to treat the matching fields as one field scores are very similar but we have a small bias towards the // more frequent field that acts as a tie-breaker internally @@ -596,7 +596,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertHitCount(searchResponse, 2L); assertFirstHit(searchResponse, hasId("ultimate2")); assertSecondHit(searchResponse, hasId("ultimate1")); - assertThat(searchResponse.getHits().hits()[0].getScore(), greaterThan(searchResponse.getHits().hits()[1].getScore())); + assertThat(searchResponse.getHits().getHits()[0].getScore(), greaterThan(searchResponse.getHits().getHits()[1].getScore())); // Test group based on numeric fields searchResponse = client().prepareSearch("test") @@ -688,7 +688,7 @@ public class MultiMatchQueryIT extends ESIntegTestCase { assertNotEquals("both documents should be on different shards", hits[0].getShard().getShardId(), hits[1].getShard().getShardId()); assertEquals("1", hits[0].getId()); assertEquals("2", hits[1].getId()); - assertThat(hits[0].getScore(), greaterThan(hits[1].score())); + assertThat(hits[0].getScore(), greaterThan(hits[1].getScore())); } private static void assertEquivalent(String query, SearchResponse left, SearchResponse right) { diff --git a/core/src/test/java/org/elasticsearch/search/query/QueryStringIT.java b/core/src/test/java/org/elasticsearch/search/query/QueryStringIT.java index 1dbeb24d4ed..05a72276362 100644 --- a/core/src/test/java/org/elasticsearch/search/query/QueryStringIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/QueryStringIT.java @@ -353,10 +353,10 @@ public class QueryStringIT extends ESIntegTestCase { } private void assertHits(SearchHits hits, String... ids) { - assertThat(hits.totalHits(), equalTo((long) ids.length)); + assertThat(hits.getTotalHits(), equalTo((long) ids.length)); Set hitIds = new HashSet<>(); for (SearchHit hit : hits.getHits()) { - hitIds.add(hit.id()); + hitIds.add(hit.getId()); } assertThat(hitIds, containsInAnyOrder(ids)); } diff --git a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java index 7cb9627cf18..05dc973f9e2 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java @@ -25,7 +25,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; @@ -98,7 +97,6 @@ import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; public class SearchQueryIT extends ESIntegTestCase { @@ -180,7 +178,7 @@ public class SearchQueryIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch().setQuery(constantScoreQuery(matchQuery("field1", "quick"))).get(); assertHitCount(searchResponse, 2L); - for (SearchHit searchHit : searchResponse.getHits().hits()) { + for (SearchHit searchHit : searchResponse.getHits().getHits()) { assertSearchHit(searchHit, hasScore(1.0f)); } @@ -188,18 +186,18 @@ public class SearchQueryIT extends ESIntegTestCase { boolQuery().must(matchAllQuery()).must( constantScoreQuery(matchQuery("field1", "quick")).boost(1.0f + random().nextFloat()))).get(); assertHitCount(searchResponse, 2L); - assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).score())); + assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).getScore())); client().prepareSearch("test").setQuery(constantScoreQuery(matchQuery("field1", "quick")).boost(1.0f + random().nextFloat())).get(); assertHitCount(searchResponse, 2L); - assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).score())); + assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).getScore())); searchResponse = client().prepareSearch("test").setQuery( constantScoreQuery(boolQuery().must(matchAllQuery()).must( constantScoreQuery(matchQuery("field1", "quick")).boost(1.0f + (random.nextBoolean()? 0.0f : random.nextFloat()))))).get(); assertHitCount(searchResponse, 2L); - assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).score())); - for (SearchHit searchHit : searchResponse.getHits().hits()) { + assertFirstHit(searchResponse, hasScore(searchResponse.getHits().getAt(1).getScore())); + for (SearchHit searchHit : searchResponse.getHits().getHits()) { assertSearchHit(searchHit, hasScore(1.0f)); } @@ -215,7 +213,7 @@ public class SearchQueryIT extends ESIntegTestCase { for (int i = 0; i < queryRounds; i++) { MatchQueryBuilder matchQuery = matchQuery("f", English.intToEnglish(between(0, num))); searchResponse = client().prepareSearch("test_1").setQuery(constantScoreQuery(matchQuery)).setSize(num).get(); - long totalHits = searchResponse.getHits().totalHits(); + long totalHits = searchResponse.getHits().getTotalHits(); SearchHits hits = searchResponse.getHits(); for (SearchHit searchHit : hits) { assertSearchHit(searchHit, hasScore(1.0f)); @@ -224,9 +222,9 @@ public class SearchQueryIT extends ESIntegTestCase { boolQuery().must(matchAllQuery()).must( constantScoreQuery(matchQuery).boost(1.0f + (random.nextBoolean()? 0.0f : random.nextFloat())))).setSize(num).get(); hits = searchResponse.getHits(); - assertThat(hits.totalHits(), equalTo(totalHits)); + assertThat(hits.getTotalHits(), equalTo(totalHits)); if (totalHits > 1) { - float expected = hits.getAt(0).score(); + float expected = hits.getAt(0).getScore(); for (SearchHit searchHit : hits) { assertSearchHit(searchHit, hasScore(expected)); } @@ -249,8 +247,8 @@ public class SearchQueryIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test").setQuery( boolQuery().must(matchAllQuery()).must(constantScoreQuery(matchAllQuery()))).get(); assertHitCount(searchResponse, 2L); - assertThat((double)searchResponse.getHits().getAt(0).score(), closeTo(2.0, 0.1)); - assertThat((double)searchResponse.getHits().getAt(1).score(),closeTo(2.0, 0.1)); + assertThat((double)searchResponse.getHits().getAt(0).getScore(), closeTo(2.0, 0.1)); + assertThat((double)searchResponse.getHits().getAt(1).getScore(),closeTo(2.0, 0.1)); } } @@ -270,7 +268,7 @@ public class SearchQueryIT extends ESIntegTestCase { assertThirdHit(searchResponse, hasId("3")); searchResponse = client().prepareSearch().setQuery(commonTermsQuery("field1", "the quick brown").cutoffFrequency(3).lowFreqOperator(Operator.AND)).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); assertFirstHit(searchResponse, hasId("1")); assertSecondHit(searchResponse, hasId("2")); @@ -357,7 +355,7 @@ public class SearchQueryIT extends ESIntegTestCase { assertThirdHit(searchResponse, hasId("3")); searchResponse = client().prepareSearch().setQuery(commonTermsQuery("field1", "the fast brown").cutoffFrequency(3).lowFreqOperator(Operator.AND)).get(); - assertThat(searchResponse.getHits().totalHits(), equalTo(2L)); + assertThat(searchResponse.getHits().getTotalHits(), equalTo(2L)); assertFirstHit(searchResponse, hasId("1")); assertSecondHit(searchResponse, hasId("2")); @@ -959,14 +957,14 @@ public class SearchQueryIT extends ESIntegTestCase { assertHitCount(searchResponse, 2L); assertFirstHit(searchResponse, hasId("1")); assertSecondHit(searchResponse, hasId("2")); - assertThat((double)searchResponse.getHits().getAt(0).score(), closeTo(boost * searchResponse.getHits().getAt(1).score(), .1)); + assertThat((double)searchResponse.getHits().getAt(0).getScore(), closeTo(boost * searchResponse.getHits().getAt(1).getScore(), .1)); searchResponse = client().prepareSearch() .setQuery(queryStringQuery("\"phrase match\"").field("important", boost).field("less_important").useDisMax(false)).get(); assertHitCount(searchResponse, 2L); assertFirstHit(searchResponse, hasId("1")); assertSecondHit(searchResponse, hasId("2")); - assertThat((double)searchResponse.getHits().getAt(0).score(), closeTo(boost * searchResponse.getHits().getAt(1).score(), .1)); + assertThat((double)searchResponse.getHits().getAt(0).getScore(), closeTo(boost * searchResponse.getHits().getAt(1).getScore(), .1)); } public void testSpecialRangeSyntaxInQueryString() { @@ -1190,27 +1188,27 @@ public class SearchQueryIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch().setQuery(idsQuery("type1", "type2").addIds("1", "2")).get(); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); searchResponse = client().prepareSearch().setQuery(idsQuery().addIds("1")).get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); searchResponse = client().prepareSearch().setQuery(idsQuery().addIds("1", "2")).get(); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); searchResponse = client().prepareSearch().setQuery(idsQuery("type1").addIds("1", "2")).get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); searchResponse = client().prepareSearch().setQuery(idsQuery(Strings.EMPTY_ARRAY).addIds("1")).get(); assertHitCount(searchResponse, 1L); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); searchResponse = client().prepareSearch().setQuery(idsQuery("type1", "type2", "type3").addIds("1", "2", "3", "4")).get(); assertHitCount(searchResponse, 2L); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); } public void testNumericTermsAndRanges() throws Exception { @@ -1879,8 +1877,8 @@ public class SearchQueryIT extends ESIntegTestCase { .setSearchType(SearchType.DFS_QUERY_THEN_FETCH) .setQuery(QueryBuilders.queryStringQuery("xyz").boost(100)) .get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); float first = response.getHits().getAt(0).getScore(); for (int i = 0; i < 100; i++) { @@ -1889,8 +1887,8 @@ public class SearchQueryIT extends ESIntegTestCase { .setQuery(QueryBuilders.queryStringQuery("xyz").boost(100)) .get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); float actual = response.getHits().getAt(0).getScore(); assertThat(i + " expected: " + first + " actual: " + actual, Float.compare(first, actual), equalTo(0)); } diff --git a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java index 1bec4bb20bf..c1b930bcc3f 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java @@ -21,11 +21,9 @@ package org.elasticsearch.search.query; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -592,10 +590,10 @@ public class SimpleQueryStringIT extends ESIntegTestCase { } private void assertHits(SearchHits hits, String... ids) { - assertThat(hits.totalHits(), equalTo((long) ids.length)); + assertThat(hits.getTotalHits(), equalTo((long) ids.length)); Set hitIds = new HashSet<>(); for (SearchHit hit : hits.getHits()) { - hitIds.add(hit.id()); + hitIds.add(hit.getId()); } assertThat(hitIds, containsInAnyOrder(ids)); } diff --git a/core/src/test/java/org/elasticsearch/search/scriptfilter/ScriptQuerySearchIT.java b/core/src/test/java/org/elasticsearch/search/scriptfilter/ScriptQuerySearchIT.java index 8d235f0a78f..45e374b8697 100644 --- a/core/src/test/java/org/elasticsearch/search/scriptfilter/ScriptQuerySearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/scriptfilter/ScriptQuerySearchIT.java @@ -131,9 +131,9 @@ public class ScriptQuerySearchIT extends ESIntegTestCase { new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['binaryData'].get(0).length", emptyMap())) .get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).id(), equalTo("2")); - assertThat(response.getHits().getAt(0).fields().get("sbinaryData").values().get(0), equalTo(16)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("2")); + assertThat(response.getHits().getAt(0).getFields().get("sbinaryData").getValues().get(0), equalTo(16)); } @@ -178,11 +178,11 @@ public class ScriptQuerySearchIT extends ESIntegTestCase { new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['num1'].value", Collections.emptyMap())) .get(); - assertThat(response.getHits().totalHits(), equalTo(2L)); - assertThat(response.getHits().getAt(0).id(), equalTo("2")); - assertThat(response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(2.0)); - assertThat(response.getHits().getAt(1).id(), equalTo("3")); - assertThat(response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(3.0)); + assertThat(response.getHits().getTotalHits(), equalTo(2L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("2")); + assertThat(response.getHits().getAt(0).getFields().get("sNum1").getValues().get(0), equalTo(2.0)); + assertThat(response.getHits().getAt(1).getId(), equalTo("3")); + assertThat(response.getHits().getAt(1).getFields().get("sNum1").getValues().get(0), equalTo(3.0)); Map params = new HashMap<>(); params.put("param1", 2); @@ -196,9 +196,9 @@ public class ScriptQuerySearchIT extends ESIntegTestCase { new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['num1'].value", Collections.emptyMap())) .get(); - assertThat(response.getHits().totalHits(), equalTo(1L)); - assertThat(response.getHits().getAt(0).id(), equalTo("3")); - assertThat(response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(3.0)); + assertThat(response.getHits().getTotalHits(), equalTo(1L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("3")); + assertThat(response.getHits().getAt(0).getFields().get("sNum1").getValues().get(0), equalTo(3.0)); params = new HashMap<>(); params.put("param1", -1); @@ -211,13 +211,13 @@ public class ScriptQuerySearchIT extends ESIntegTestCase { new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['num1'].value", Collections.emptyMap())) .get(); - assertThat(response.getHits().totalHits(), equalTo(3L)); - assertThat(response.getHits().getAt(0).id(), equalTo("1")); - assertThat(response.getHits().getAt(0).fields().get("sNum1").values().get(0), equalTo(1.0)); - assertThat(response.getHits().getAt(1).id(), equalTo("2")); - assertThat(response.getHits().getAt(1).fields().get("sNum1").values().get(0), equalTo(2.0)); - assertThat(response.getHits().getAt(2).id(), equalTo("3")); - assertThat(response.getHits().getAt(2).fields().get("sNum1").values().get(0), equalTo(3.0)); + assertThat(response.getHits().getTotalHits(), equalTo(3L)); + assertThat(response.getHits().getAt(0).getId(), equalTo("1")); + assertThat(response.getHits().getAt(0).getFields().get("sNum1").getValues().get(0), equalTo(1.0)); + assertThat(response.getHits().getAt(1).getId(), equalTo("2")); + assertThat(response.getHits().getAt(1).getFields().get("sNum1").getValues().get(0), equalTo(2.0)); + assertThat(response.getHits().getAt(2).getId(), equalTo("3")); + assertThat(response.getHits().getAt(2).getFields().get("sNum1").getValues().get(0), equalTo(3.0)); } private static AtomicInteger scriptCounter = new AtomicInteger(0); diff --git a/core/src/test/java/org/elasticsearch/search/scroll/DuelScrollIT.java b/core/src/test/java/org/elasticsearch/search/scroll/DuelScrollIT.java index 60778a4eb07..31fcfa7155c 100644 --- a/core/src/test/java/org/elasticsearch/search/scroll/DuelScrollIT.java +++ b/core/src/test/java/org/elasticsearch/search/scroll/DuelScrollIT.java @@ -51,7 +51,7 @@ public class DuelScrollIT extends ESIntegTestCase { .setSize(context.numDocs).get(); assertNoFailures(control); SearchHits sh = control.getHits(); - assertThat(sh.totalHits(), equalTo((long) context.numDocs)); + assertThat(sh.getTotalHits(), equalTo((long) context.numDocs)); assertThat(sh.getHits().length, equalTo(context.numDocs)); SearchResponse searchScrollResponse = client().prepareSearch("index") @@ -62,11 +62,11 @@ public class DuelScrollIT extends ESIntegTestCase { assertNoFailures(searchScrollResponse); assertThat(searchScrollResponse.getHits().getTotalHits(), equalTo((long) context.numDocs)); - assertThat(searchScrollResponse.getHits().hits().length, equalTo(context.scrollRequestSize)); + assertThat(searchScrollResponse.getHits().getHits().length, equalTo(context.scrollRequestSize)); int counter = 0; for (SearchHit hit : searchScrollResponse.getHits()) { - assertThat(hit.sortValues()[0], equalTo(sh.getAt(counter++).sortValues()[0])); + assertThat(hit.getSortValues()[0], equalTo(sh.getAt(counter++).getSortValues()[0])); } int iter = 1; @@ -75,7 +75,7 @@ public class DuelScrollIT extends ESIntegTestCase { searchScrollResponse = client().prepareSearchScroll(scrollId).setScroll("10m").get(); assertNoFailures(searchScrollResponse); assertThat(searchScrollResponse.getHits().getTotalHits(), equalTo((long) context.numDocs)); - if (searchScrollResponse.getHits().hits().length == 0) { + if (searchScrollResponse.getHits().getHits().length == 0) { break; } @@ -86,9 +86,9 @@ public class DuelScrollIT extends ESIntegTestCase { } else { expectedLength = context.scrollRequestSize - (scrollSlice - context.numDocs); } - assertThat(searchScrollResponse.getHits().hits().length, equalTo(expectedLength)); + assertThat(searchScrollResponse.getHits().getHits().length, equalTo(expectedLength)); for (SearchHit hit : searchScrollResponse.getHits()) { - assertThat(hit.sortValues()[0], equalTo(sh.getAt(counter++).sortValues()[0])); + assertThat(hit.getSortValues()[0], equalTo(sh.getAt(counter++).getSortValues()[0])); } scrollId = searchScrollResponse.getScrollId(); } @@ -235,15 +235,15 @@ public class DuelScrollIT extends ESIntegTestCase { assertNoFailures(scroll); assertEquals(control.getHits().getTotalHits(), scroll.getHits().getTotalHits()); assertEquals(control.getHits().getMaxScore(), scroll.getHits().getMaxScore(), 0.01f); - if (scroll.getHits().hits().length == 0) { + if (scroll.getHits().getHits().length == 0) { break; } - for (int i = 0; i < scroll.getHits().hits().length; ++i) { + for (int i = 0; i < scroll.getHits().getHits().length; ++i) { SearchHit controlHit = control.getHits().getAt(scrollDocs + i); SearchHit scrollHit = scroll.getHits().getAt(i); assertEquals(controlHit.getId(), scrollHit.getId()); } - scrollDocs += scroll.getHits().hits().length; + scrollDocs += scroll.getHits().getHits().length; scroll = client().prepareSearchScroll(scroll.getScrollId()).setScroll("10m").get(); } assertEquals(control.getHits().getTotalHits(), scrollDocs); diff --git a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java index 1f368cb35da..c887b20e11f 100644 --- a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java +++ b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java @@ -85,9 +85,9 @@ public class SearchScrollIT extends ESIntegTestCase { long counter = 0; assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(35)); + assertThat(searchResponse.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()) @@ -95,9 +95,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(35)); + assertThat(searchResponse.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()) @@ -105,9 +105,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(30)); + assertThat(searchResponse.getHits().getHits().length, equalTo(30)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } } finally { clearScroll(searchResponse.getScrollId()); @@ -143,9 +143,9 @@ public class SearchScrollIT extends ESIntegTestCase { long counter = 0; assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } for (int i = 0; i < 32; i++) { @@ -154,9 +154,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } } @@ -166,9 +166,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } // a the last is zero @@ -177,9 +177,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(0)); + assertThat(searchResponse.getHits().getHits().length, equalTo(0)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } } finally { @@ -198,11 +198,11 @@ public class SearchScrollIT extends ESIntegTestCase { client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(500L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().totalHits(), equalTo(500L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().totalHits(), equalTo(500L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().totalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); SearchResponse searchResponse = client().prepareSearch() .setQuery(queryStringQuery("user:kimchy")) @@ -212,20 +212,20 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); try { do { - for (SearchHit searchHit : searchResponse.getHits().hits()) { - Map map = searchHit.sourceAsMap(); + for (SearchHit searchHit : searchResponse.getHits().getHits()) { + Map map = searchHit.getSourceAsMap(); map.put("message", "update"); - client().prepareIndex("test", "tweet", searchHit.id()).setSource(map).execute().actionGet(); + client().prepareIndex("test", "tweet", searchHit.getId()).setSource(map).execute().actionGet(); } searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(2)).execute().actionGet(); - } while (searchResponse.getHits().hits().length > 0); + } while (searchResponse.getHits().getHits().length > 0); client().admin().indices().prepareRefresh().execute().actionGet(); - assertThat(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().totalHits(), equalTo(500L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().totalHits(), equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().totalHits(), equalTo(500L)); - assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().totalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "test")).execute().actionGet().getHits().getTotalHits(), equalTo(0L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); + assertThat(client().prepareSearch().setSize(0).setQuery(termQuery("message", "update")).execute().actionGet().getHits().getTotalHits(), equalTo(500L)); } finally { clearScroll(searchResponse.getScrollId()); } @@ -263,15 +263,15 @@ public class SearchScrollIT extends ESIntegTestCase { long counter2 = 0; assertThat(searchResponse1.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse1.getHits().hits().length, equalTo(35)); + assertThat(searchResponse1.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse1.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter1++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter1++)); } assertThat(searchResponse2.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse2.getHits().hits().length, equalTo(35)); + assertThat(searchResponse2.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse2.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter2++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter2++)); } searchResponse1 = client().prepareSearchScroll(searchResponse1.getScrollId()) @@ -283,15 +283,15 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse1.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse1.getHits().hits().length, equalTo(35)); + assertThat(searchResponse1.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse1.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter1++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter1++)); } assertThat(searchResponse2.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse2.getHits().hits().length, equalTo(35)); + assertThat(searchResponse2.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse2.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter2++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter2++)); } ClearScrollResponse clearResponse = client().prepareClearScroll() @@ -369,15 +369,15 @@ public class SearchScrollIT extends ESIntegTestCase { long counter2 = 0; assertThat(searchResponse1.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse1.getHits().hits().length, equalTo(35)); + assertThat(searchResponse1.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse1.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter1++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter1++)); } assertThat(searchResponse2.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse2.getHits().hits().length, equalTo(35)); + assertThat(searchResponse2.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse2.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter2++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter2++)); } searchResponse1 = client().prepareSearchScroll(searchResponse1.getScrollId()) @@ -389,15 +389,15 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse1.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse1.getHits().hits().length, equalTo(35)); + assertThat(searchResponse1.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse1.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter1++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter1++)); } assertThat(searchResponse2.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse2.getHits().hits().length, equalTo(35)); + assertThat(searchResponse2.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse2.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter2++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter2++)); } ClearScrollResponse clearResponse = client().prepareClearScroll().addScrollId("_all") @@ -505,9 +505,9 @@ public class SearchScrollIT extends ESIntegTestCase { .execute().actionGet(); long counter = 0; assertThat(searchResponse.getHits().getTotalHits(), equalTo(100L)); - assertThat(searchResponse.getHits().hits().length, equalTo(35)); + assertThat(searchResponse.getHits().getHits().length, equalTo(35)); for (SearchHit hit : searchResponse.getHits()) { - assertThat(((Number) hit.sortValues()[0]).longValue(), equalTo(counter++)); + assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } if (randomBoolean()) { client().admin().indices().prepareClose("test").get(); diff --git a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollWithFailingNodesIT.java b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollWithFailingNodesIT.java index 5b34dbc55af..0b30048826a 100644 --- a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollWithFailingNodesIT.java +++ b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollWithFailingNodesIT.java @@ -76,12 +76,12 @@ public class SearchScrollWithFailingNodesIT extends ESIntegTestCase { assertAllSuccessful(searchResponse); long numHits = 0; do { - numHits += searchResponse.getHits().hits().length; + numHits += searchResponse.getHits().getHits().length; searchResponse = client() .prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(1)) .get(); assertAllSuccessful(searchResponse); - } while (searchResponse.getHits().hits().length > 0); + } while (searchResponse.getHits().getHits().length > 0); assertThat(numHits, equalTo(100L)); clearScroll("_all"); @@ -96,12 +96,12 @@ public class SearchScrollWithFailingNodesIT extends ESIntegTestCase { numHits = 0; int numberOfSuccessfulShards = searchResponse.getSuccessfulShards(); do { - numHits += searchResponse.getHits().hits().length; + numHits += searchResponse.getHits().getHits().length; searchResponse = client() .prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(1)) .get(); assertThat(searchResponse.getSuccessfulShards(), equalTo(numberOfSuccessfulShards)); - } while (searchResponse.getHits().hits().length > 0); + } while (searchResponse.getHits().getHits().length > 0); assertThat(numHits, greaterThan(0L)); clearScroll(searchResponse.getScrollId()); diff --git a/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java b/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java index 08c661a130b..185e50400f6 100644 --- a/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java +++ b/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java @@ -154,8 +154,8 @@ public class SearchAfterIT extends ESIntegTestCase { .get(); assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(2L)); assertThat(searchResponse.getHits().getHits().length, Matchers.equalTo(1)); - assertThat(searchResponse.getHits().getHits()[0].sourceAsMap().get("field1"), Matchers.equalTo(100)); - assertThat(searchResponse.getHits().getHits()[0].sourceAsMap().get("field2"), Matchers.equalTo("toto")); + assertThat(searchResponse.getHits().getHits()[0].getSourceAsMap().get("field1"), Matchers.equalTo(100)); + assertThat(searchResponse.getHits().getHits()[0].getSourceAsMap().get("field2"), Matchers.equalTo("toto")); } public void testWithSimpleTypes() throws Exception { @@ -262,9 +262,9 @@ public class SearchAfterIT extends ESIntegTestCase { SearchResponse searchResponse = req.get(); for (SearchHit hit : searchResponse.getHits()) { List toCompare = convertSortValues(documents.get(offset++)); - assertThat(LST_COMPARATOR.compare(toCompare, Arrays.asList(hit.sortValues())), equalTo(0)); + assertThat(LST_COMPARATOR.compare(toCompare, Arrays.asList(hit.getSortValues())), equalTo(0)); } - sortValues = searchResponse.getHits().hits()[searchResponse.getHits().hits().length-1].getSortValues(); + sortValues = searchResponse.getHits().getHits()[searchResponse.getHits().getHits().length-1].getSortValues(); } } diff --git a/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java index 76595765558..a8911afe1fa 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java +++ b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java @@ -103,9 +103,9 @@ public class FieldSortIT extends ESIntegTestCase { logClusterState(); assertSearchResponse(searchResponse); - for (int j = 1; j < searchResponse.getHits().hits().length; j++) { - Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); - Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); + for (int j = 1; j < searchResponse.getHits().getHits().length; j++) { + Number current = (Number) searchResponse.getHits().getHits()[j].getSourceAsMap().get("entry"); + Number previous = (Number) searchResponse.getHits().getHits()[j-1].getSourceAsMap().get("entry"); assertThat(searchResponse.toString(), current.intValue(), lessThan(previous.intValue())); } @@ -116,9 +116,9 @@ public class FieldSortIT extends ESIntegTestCase { logClusterState(); assertSearchResponse(searchResponse); - for (int j = 1; j < searchResponse.getHits().hits().length; j++) { - Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); - Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); + for (int j = 1; j < searchResponse.getHits().getHits().length; j++) { + Number current = (Number) searchResponse.getHits().getHits()[j].getSourceAsMap().get("entry"); + Number previous = (Number) searchResponse.getHits().getHits()[j-1].getSourceAsMap().get("entry"); assertThat(searchResponse.toString(), current.intValue(), greaterThan(previous.intValue())); } } @@ -163,12 +163,12 @@ public class FieldSortIT extends ESIntegTestCase { .addSort(new FieldSortBuilder("timeUpdated").order(SortOrder.ASC).unmappedType("date")) .setSize(scaledRandomIntBetween(1, docs)).get(); assertSearchResponse(searchResponse); - for (int j = 0; j < searchResponse.getHits().hits().length; j++) { + for (int j = 0; j < searchResponse.getHits().getHits().length; j++) { assertThat(searchResponse.toString() + "\n vs. \n" + allDocsResponse.toString(), - searchResponse.getHits().hits()[j].getId(), - equalTo(allDocsResponse.getHits().hits()[j].getId())); + searchResponse.getHits().getHits()[j].getId(), + equalTo(allDocsResponse.getHits().getHits()[j].getId())); } } @@ -260,14 +260,14 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("dense_bytes", SortOrder.ASC).execute().actionGet(); assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) numDocs)); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); Set> entrySet = denseBytes.entrySet(); Iterator> iterator = entrySet.iterator(); for (int i = 0; i < size; i++) { assertThat(iterator.hasNext(), equalTo(true)); Entry next = iterator.next(); - assertThat("pos: " + i, searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); + assertThat("pos: " + i, searchResponse.getHits().getAt(i).getId(), equalTo(next.getValue())); + assertThat(searchResponse.getHits().getAt(i).getSortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); } } if (!sparseBytes.isEmpty()) { @@ -277,14 +277,14 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) sparseBytes.size())); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); Set> entrySet = sparseBytes.entrySet(); Iterator> iterator = entrySet.iterator(); for (int i = 0; i < size; i++) { assertThat(iterator.hasNext(), equalTo(true)); Entry next = iterator.next(); - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(next.getValue())); + assertThat(searchResponse.getHits().getAt(i).getSortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); } } } @@ -300,9 +300,9 @@ public class FieldSortIT extends ESIntegTestCase { refresh(); SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0].toString(), equalTo("100")); // reindex and refresh client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); @@ -310,18 +310,18 @@ public class FieldSortIT extends ESIntegTestCase { searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0].toString(), equalTo("100")); // reindex - no refresh client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0].toString(), equalTo("100")); // force merge forceMerge(); @@ -330,16 +330,16 @@ public class FieldSortIT extends ESIntegTestCase { client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0].toString(), equalTo("100")); refresh(); searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0].toString(), equalTo("100")); } public void testScoreSortDirection() throws Exception { @@ -358,9 +358,9 @@ public class FieldSortIT extends ESIntegTestCase { QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) .execute().actionGet(); assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(0).getScore())); assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(1).getScore())); assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); searchResponse = client() @@ -369,9 +369,9 @@ public class FieldSortIT extends ESIntegTestCase { QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) .addSort("_score", SortOrder.DESC).execute().actionGet(); assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(0).getScore())); assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(1).getScore())); assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); searchResponse = client() @@ -397,18 +397,18 @@ public class FieldSortIT extends ESIntegTestCase { SearchResponse searchResponse = client().prepareSearch("test") .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))).execute().actionGet(); assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(0).getScore())); assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(1).getScore())); assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); searchResponse = client().prepareSearch("test") .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))) .addSort("_score", SortOrder.DESC).execute().actionGet(); assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(0).getScore())); assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getScore(), Matchers.lessThan(searchResponse.getHits().getAt(1).getScore())); assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); searchResponse = client().prepareSearch("test") @@ -460,14 +460,14 @@ public class FieldSortIT extends ESIntegTestCase { SearchResponse resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("tag").order(SortOrder.ASC)).execute().actionGet(); assertHitCount(resp, 4); - assertThat(resp.getHits().hits().length, equalTo(2)); + assertThat(resp.getHits().getHits().length, equalTo(2)); assertFirstHit(resp, hasId("1")); assertSecondHit(resp, hasId("2")); resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()) .addSort(SortBuilders.fieldSort("tag").order(SortOrder.DESC)).execute().actionGet(); assertHitCount(resp, 4); - assertThat(resp.getHits().hits().length, equalTo(2)); + assertThat(resp.getHits().getHits().length, equalTo(2)); assertFirstHit(resp, hasId("3")); assertSecondHit(resp, hasId("4")); } @@ -521,10 +521,10 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("str_value", SortOrder.ASC) .execute().actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(searchResponse.getHits().getAt(i).getSortValues()[0].toString(), equalTo(new String(new char[] { (char) (97 + i), (char) (97 + i) }))); } size = 1 + random.nextInt(10); @@ -535,10 +535,10 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(searchResponse.getHits().getAt(i).getSortValues()[0].toString(), equalTo(new String(new char[] { (char) (97 + (9 - i)), (char) (97 + (9 - i)) }))); } @@ -551,20 +551,20 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) i)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).byteValue(), equalTo((byte) i)); } size = 1 + random.nextInt(10); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("byte_value", SortOrder.DESC).execute() .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) (9 - i))); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).byteValue(), equalTo((byte) (9 - i))); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -575,20 +575,20 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) i)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).shortValue(), equalTo((short) i)); } size = 1 + random.nextInt(10); searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("short_value", SortOrder.DESC).execute() .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) (9 - i))); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).shortValue(), equalTo((short) (9 - i))); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -599,10 +599,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo(i)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).intValue(), equalTo(i)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -611,10 +611,10 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo((9 - i))); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).intValue(), equalTo((9 - i))); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -625,10 +625,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) i)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).longValue(), equalTo((long) i)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -637,10 +637,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10L); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) (9 - i))); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).longValue(), equalTo((long) (9 - i))); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -651,10 +651,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -663,10 +663,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -677,10 +677,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -689,10 +689,10 @@ public class FieldSortIT extends ESIntegTestCase { .actionGet(); assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); + assertThat(searchResponse.getHits().getAt(i).getId(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).getSortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); } assertNoFailures(searchResponse); @@ -741,9 +741,9 @@ public class FieldSortIT extends ESIntegTestCase { assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("2")); logger.info("--> sort with missing _last"); searchResponse = client().prepareSearch() @@ -753,9 +753,9 @@ public class FieldSortIT extends ESIntegTestCase { assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("2")); logger.info("--> sort with missing _first"); searchResponse = client().prepareSearch() @@ -765,9 +765,9 @@ public class FieldSortIT extends ESIntegTestCase { assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); } public void testSortMissingStrings() throws IOException { @@ -815,9 +815,9 @@ public class FieldSortIT extends ESIntegTestCase { assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("2")); logger.info("--> sort with missing _last"); searchResponse = client().prepareSearch() @@ -827,9 +827,9 @@ public class FieldSortIT extends ESIntegTestCase { assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("2")); logger.info("--> sort with missing _first"); searchResponse = client().prepareSearch() @@ -839,9 +839,9 @@ public class FieldSortIT extends ESIntegTestCase { assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); logger.info("--> sort with missing b"); searchResponse = client().prepareSearch() @@ -851,9 +851,9 @@ public class FieldSortIT extends ESIntegTestCase { assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); } public void testIgnoreUnmapped() throws Exception { @@ -935,16 +935,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(-4L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).longValue(), equalTo(-4L)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).longValue(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(7L)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).longValue(), equalTo(7L)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -953,16 +953,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(20L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).longValue(), equalTo(20L)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(10L)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).longValue(), equalTo(10L)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).longValue(), equalTo(3L)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -971,16 +971,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(53L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).longValue(), equalTo(53L)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(24L)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).longValue(), equalTo(24L)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).longValue(), equalTo(2L)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -989,16 +989,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(13L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).longValue(), equalTo(13L)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(6L)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).longValue(), equalTo(6L)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).longValue(), equalTo(1L)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1007,16 +1007,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(13L)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).longValue(), equalTo(13L)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(7L)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).longValue(), equalTo(7L)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).longValue(), equalTo(2L)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1025,16 +1025,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(-4)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(7)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1043,16 +1043,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(20)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(10)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(3)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1061,16 +1061,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(-4)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(7)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1079,16 +1079,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(20)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(10)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(3)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1097,16 +1097,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(-4)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(1)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(7)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1115,16 +1115,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).intValue(), equalTo(20)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).intValue(), equalTo(10)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).intValue(), equalTo(3)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1133,16 +1133,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(-4f)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).floatValue(), equalTo(-4f)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(1f)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).floatValue(), equalTo(1f)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(7f)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).floatValue(), equalTo(7f)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1151,16 +1151,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(20f)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).floatValue(), equalTo(20f)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(10f)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).floatValue(), equalTo(10f)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(3f)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).floatValue(), equalTo(3f)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1169,16 +1169,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(-4d)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), equalTo(-4d)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(1d)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), equalTo(1d)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(7d)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), equalTo(7d)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1187,16 +1187,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(20d)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), equalTo(20d)); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(10d)); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), equalTo(10d)); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(3d)); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), equalTo(3d)); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1205,16 +1205,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("!4")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(3))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("!4")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0], equalTo("01")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0], equalTo("01")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0], equalTo("07")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(2))); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0], equalTo("07")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -1223,16 +1223,16 @@ public class FieldSortIT extends ESIntegTestCase { .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("20")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("20")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0], equalTo("10")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0], equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0], equalTo("03")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0], equalTo("03")); } public void testSortOnRareField() throws IOException { @@ -1252,11 +1252,11 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("string_values", SortOrder.DESC) .execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("10")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("10")); client().prepareIndex("test", "type1", Integer.toString(2)).setSource(jsonBuilder().startObject() .array("string_values", "11", "15", "20", "07") @@ -1274,13 +1274,13 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("string_values", SortOrder.DESC) .execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(2)); + assertThat(searchResponse.getHits().getHits().length, equalTo(2)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("20")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("20")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0], equalTo("10")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0], equalTo("10")); client().prepareIndex("test", "type1", Integer.toString(3)).setSource(jsonBuilder().startObject() @@ -1299,16 +1299,16 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("string_values", SortOrder.DESC) .execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("20")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("20")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0], equalTo("10")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0], equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0], equalTo("03")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0], equalTo("03")); for (int i = 0; i < 15; i++) { client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() @@ -1323,16 +1323,16 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("string_values", SortOrder.DESC) .execute().actionGet(); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); + assertThat(searchResponse.getHits().getHits().length, equalTo(3)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0], equalTo("20")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo(Integer.toString(2))); + assertThat(searchResponse.getHits().getAt(0).getSortValues()[0], equalTo("20")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0], equalTo("10")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo(Integer.toString(1))); + assertThat(searchResponse.getHits().getAt(1).getSortValues()[0], equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0], equalTo("03")); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo(Integer.toString(3))); + assertThat(searchResponse.getHits().getAt(2).getSortValues()[0], equalTo("03")); } public void testSortMetaField() throws Exception { @@ -1353,10 +1353,10 @@ public class FieldSortIT extends ESIntegTestCase { .addSort("_uid", order) .execute().actionGet(); assertNoFailures(searchResponse); - SearchHit[] hits = searchResponse.getHits().hits(); + SearchHit[] hits = searchResponse.getHits().getHits(); BytesRef previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; for (int i = 0; i < hits.length; ++i) { - final BytesRef uid = new BytesRef(Uid.createUid(hits[i].type(), hits[i].id())); + final BytesRef uid = new BytesRef(Uid.createUid(hits[i].getType(), hits[i].getId())); assertThat(previous, order == SortOrder.ASC ? lessThan(uid) : greaterThan(uid)); previous = uid; } @@ -1404,7 +1404,7 @@ public class FieldSortIT extends ESIntegTestCase { .addSort(SortBuilders.fieldSort("nested.foo").setNestedPath("nested").order(SortOrder.DESC)) .execute().actionGet(); assertNoFailures(searchResponse); - SearchHit[] hits = searchResponse.getHits().hits(); + SearchHit[] hits = searchResponse.getHits().getHits(); for (int i = 0; i < hits.length; ++i) { assertThat(hits[i].getSortValues().length, is(1)); assertThat(hits[i].getSortValues()[0], is("bar")); @@ -1417,7 +1417,7 @@ public class FieldSortIT extends ESIntegTestCase { .addSort(SortBuilders.fieldSort("nested.foo.sub").setNestedPath("nested").order(SortOrder.DESC)) .execute().actionGet(); assertNoFailures(searchResponse); - hits = searchResponse.getHits().hits(); + hits = searchResponse.getHits().getHits(); for (int i = 0; i < hits.length; ++i) { assertThat(hits[i].getSortValues().length, is(1)); assertThat(hits[i].getSortValues()[0], is("bar bar")); @@ -1450,12 +1450,12 @@ public class FieldSortIT extends ESIntegTestCase { SearchResponse singleShardResponse = client().prepareSearch("test2").setFrom(from).setSize(size).addSort(sortField, order).get(); assertNoFailures(singleShardResponse); - assertThat(multiShardResponse.getHits().totalHits(), equalTo(singleShardResponse.getHits().totalHits())); + assertThat(multiShardResponse.getHits().getTotalHits(), equalTo(singleShardResponse.getHits().getTotalHits())); assertThat(multiShardResponse.getHits().getHits().length, equalTo(singleShardResponse.getHits().getHits().length)); for (int i = 0; i < multiShardResponse.getHits().getHits().length; i++) { - assertThat(multiShardResponse.getHits().getAt(i).sortValues()[0], - equalTo(singleShardResponse.getHits().getAt(i).sortValues()[0])); - assertThat(multiShardResponse.getHits().getAt(i).id(), equalTo(singleShardResponse.getHits().getAt(i).id())); + assertThat(multiShardResponse.getHits().getAt(i).getSortValues()[0], + equalTo(singleShardResponse.getHits().getAt(i).getSortValues()[0])); + assertThat(multiShardResponse.getHits().getAt(i).getId(), equalTo(singleShardResponse.getHits().getAt(i).getId())); } } @@ -1473,7 +1473,7 @@ public class FieldSortIT extends ESIntegTestCase { .addSort(SortBuilders.fieldSort("ip")) .get(); assertSearchResponse(response); - assertEquals(2, response.getHits().totalHits()); + assertEquals(2, response.getHits().getTotalHits()); assertArrayEquals(new String[] {"192.168.1.7"}, response.getHits().getAt(0).getSortValues()); assertArrayEquals(new String[] {"2001:db8::ff00:42:8329"}, @@ -1484,8 +1484,8 @@ public class FieldSortIT extends ESIntegTestCase { .searchAfter(new Object[] {"192.168.1.7"}) .get(); assertSearchResponse(response); - assertEquals(2, response.getHits().totalHits()); - assertEquals(1, response.getHits().hits().length); + assertEquals(2, response.getHits().getTotalHits()); + assertEquals(1, response.getHits().getHits().length); assertArrayEquals(new String[] {"2001:db8::ff00:42:8329"}, response.getHits().getAt(0).getSortValues()); } diff --git a/core/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java b/core/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java index d6863d03931..a1cc9b4d4dc 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java +++ b/core/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java @@ -106,11 +106,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "1", "2", "3", "4", "5"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); // Order: Asc, Mode: max searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) @@ -119,11 +119,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "1", "2", "4", "3", "5"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); // Order: Desc searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) @@ -132,11 +132,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "5", "3", "4", "2", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); // Order: Desc, Mode: min searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) @@ -145,11 +145,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "5", "4", "3", "2", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.geoDistanceSort("locations", 40.7143528, -74.0059731).sortMode(SortMode.AVG).order(SortOrder.ASC)) @@ -157,11 +157,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "1", "2", "4", "3", "5"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1157d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(2874d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(5301d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1157d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(2874d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(5301d, 10d)); searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) .addSort(SortBuilders.geoDistanceSort("locations", 40.7143528, -74.0059731).sortMode(SortMode.AVG).order(SortOrder.DESC)) @@ -169,11 +169,11 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 5); assertOrderedSearchHits(searchResponse, "5", "3", "4", "2", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(4).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(421.2d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(4).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); try { client().prepareSearch("test").setQuery(matchAllQuery()) @@ -215,8 +215,8 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 2); assertOrderedSearchHits(searchResponse, "1", "2"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); // Order: Desc searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) @@ -226,8 +226,8 @@ public class GeoDistanceIT extends ESIntegTestCase { // Doc with missing geo point is first, is consistent with 0.20.x assertHitCount(searchResponse, 2); assertOrderedSearchHits(searchResponse, "2", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(5286d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(5286d, 10d)); } public void testDistanceSortingNestedFields() throws Exception { @@ -285,10 +285,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "1", "2", "3", "4"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); // Order: Asc, Mode: max searchResponse = client() @@ -298,10 +298,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "1", "3", "2", "4"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); // Order: Desc searchResponse = client().prepareSearch("companies").setQuery(matchAllQuery()).addSort(SortBuilders @@ -310,10 +310,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "4", "2", "3", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(5286.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1258.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); // Order: Desc, Mode: min searchResponse = client() @@ -323,10 +323,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "4", "3", "2", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(2029.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(1055.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(462.1d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); searchResponse = client() .prepareSearch("companies").setQuery(matchAllQuery()).addSort(SortBuilders.geoDistanceSort("branches.location", @@ -335,10 +335,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "1", "3", "2", "4"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); searchResponse = client().prepareSearch("companies") .setQuery(matchAllQuery()).addSort(SortBuilders.geoDistanceSort("branches.location", 40.7143528, -74.0059731) @@ -347,10 +347,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertOrderedSearchHits(searchResponse, "4", "2", "3", "1"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), closeTo(0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(5301.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), closeTo(2874.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), closeTo(1157.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), closeTo(0d, 10d)); searchResponse = client().prepareSearch("companies").setQuery(matchAllQuery()) .addSort(SortBuilders.geoDistanceSort("branches.location", 40.7143528, -74.0059731) @@ -360,10 +360,10 @@ public class GeoDistanceIT extends ESIntegTestCase { assertHitCount(searchResponse, 4); assertFirstHit(searchResponse, hasId("4")); assertSearchHits(searchResponse, "1", "2", "3", "4"); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(((Number) searchResponse.getHits().getAt(3).sortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(((Number) searchResponse.getHits().getAt(0).getSortValues()[0]).doubleValue(), closeTo(8572.0d, 10d)); + assertThat(((Number) searchResponse.getHits().getAt(1).getSortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(((Number) searchResponse.getHits().getAt(2).getSortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(((Number) searchResponse.getHits().getAt(3).getSortValues()[0]).doubleValue(), equalTo(Double.POSITIVE_INFINITY)); try { client().prepareSearch("companies").setQuery(matchAllQuery()) diff --git a/core/src/test/java/org/elasticsearch/search/sort/SimpleSortIT.java b/core/src/test/java/org/elasticsearch/search/sort/SimpleSortIT.java index d73da7e5076..50a436c7b27 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/SimpleSortIT.java +++ b/core/src/test/java/org/elasticsearch/search/sort/SimpleSortIT.java @@ -192,13 +192,13 @@ public class SimpleSortIT extends ESIntegTestCase { .get(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat(searchHit.id(), equalTo(Integer.toString(i))); + assertThat(searchHit.getId(), equalTo(Integer.toString(i))); String expected = new String(new char[]{(char) (97 + i), (char) (97 + i)}); - assertThat(searchHit.sortValues()[0].toString(), equalTo(expected)); + assertThat(searchHit.getSortValues()[0].toString(), equalTo(expected)); } size = 1 + random.nextInt(10); @@ -209,13 +209,13 @@ public class SimpleSortIT extends ESIntegTestCase { .get(); assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); + assertThat(searchResponse.getHits().getHits().length, equalTo(size)); for (int i = 0; i < size; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat(searchHit.id(), equalTo(Integer.toString(9 - i))); + assertThat(searchHit.getId(), equalTo(Integer.toString(9 - i))); String expected = new String(new char[]{(char) (97 + (9 - i)), (char) (97 + (9 - i))}); - assertThat(searchHit.sortValues()[0].toString(), equalTo(expected)); + assertThat(searchHit.getSortValues()[0].toString(), equalTo(expected)); } assertThat(searchResponse.toString(), not(containsString("error"))); @@ -285,7 +285,7 @@ public class SimpleSortIT extends ESIntegTestCase { assertHitCount(searchResponse, 20L); for (int i = 0; i < 10; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").value(), equalTo((long) i)); + assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").getValue(), equalTo((long) i)); } // test the double values @@ -301,7 +301,7 @@ public class SimpleSortIT extends ESIntegTestCase { assertHitCount(searchResponse, 20L); for (int i = 0; i < 10; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").value(), equalTo((double) i)); + assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").getValue(), equalTo((double) i)); } // test the string values @@ -317,7 +317,7 @@ public class SimpleSortIT extends ESIntegTestCase { assertHitCount(searchResponse, 20L); for (int i = 0; i < 10; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").value(), equalTo(i)); + assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").getValue(), equalTo(i)); } // test the geopoint values @@ -334,7 +334,7 @@ public class SimpleSortIT extends ESIntegTestCase { assertHitCount(searchResponse, 20L); for (int i = 0; i < 10; i++) { SearchHit searchHit = searchResponse.getHits().getAt(i); - assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").value(), closeTo(i, GeoUtils.TOLERANCE)); + assertThat("res: " + i + " id: " + searchHit.getId(), searchHit.field("min").getValue(), closeTo(i, GeoUtils.TOLERANCE)); } } @@ -393,9 +393,9 @@ public class SimpleSortIT extends ESIntegTestCase { assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).field("id").value(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).field("id").value(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).field("id").value(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).field("id").getValue(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).field("id").getValue(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).field("id").getValue(), equalTo("2")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -406,9 +406,9 @@ public class SimpleSortIT extends ESIntegTestCase { assertNoFailures(searchResponse); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).field("id").value(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).field("id").value(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).field("id").value(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).field("id").getValue(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).field("id").getValue(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).field("id").getValue(), equalTo("2")); searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) @@ -425,9 +425,9 @@ public class SimpleSortIT extends ESIntegTestCase { assertThat(searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).field("id").value(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(1).field("id").value(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(2).field("id").value(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).field("id").getValue(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(1).field("id").getValue(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(2).field("id").getValue(), equalTo("2")); // a query with docs just with null values searchResponse = client().prepareSearch() @@ -445,7 +445,7 @@ public class SimpleSortIT extends ESIntegTestCase { assertThat(searchResponse.getFailedShards(), equalTo(0)); assertThat(searchResponse.getHits().getTotalHits(), equalTo(1L)); - assertThat(searchResponse.getHits().getAt(0).field("id").value(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).field("id").getValue(), equalTo("2")); } public void test2920() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java b/core/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java index f1a0f7da45b..460fd11fbd9 100644 --- a/core/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java +++ b/core/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java @@ -43,7 +43,7 @@ public class MetadataFetchingIT extends ESIntegTestCase { .get(); assertThat(response.getHits().getAt(0).getId(), nullValue()); assertThat(response.getHits().getAt(0).getType(), nullValue()); - assertThat(response.getHits().getAt(0).sourceAsString(), nullValue()); + assertThat(response.getHits().getAt(0).getSourceAsString(), nullValue()); response = client() .prepareSearch("test") @@ -51,7 +51,7 @@ public class MetadataFetchingIT extends ESIntegTestCase { .get(); assertThat(response.getHits().getAt(0).getId(), nullValue()); assertThat(response.getHits().getAt(0).getType(), nullValue()); - assertThat(response.getHits().getAt(0).sourceAsString(), nullValue()); + assertThat(response.getHits().getAt(0).getSourceAsString(), nullValue()); } public void testWithRouting() { @@ -69,7 +69,7 @@ public class MetadataFetchingIT extends ESIntegTestCase { assertThat(response.getHits().getAt(0).getId(), nullValue()); assertThat(response.getHits().getAt(0).getType(), nullValue()); assertThat(response.getHits().getAt(0).field("_routing"), nullValue()); - assertThat(response.getHits().getAt(0).sourceAsString(), nullValue()); + assertThat(response.getHits().getAt(0).getSourceAsString(), nullValue()); response = client() .prepareSearch("test") @@ -77,7 +77,7 @@ public class MetadataFetchingIT extends ESIntegTestCase { .get(); assertThat(response.getHits().getAt(0).getId(), nullValue()); assertThat(response.getHits().getAt(0).getType(), nullValue()); - assertThat(response.getHits().getAt(0).sourceAsString(), nullValue()); + assertThat(response.getHits().getAt(0).getSourceAsString(), nullValue()); } public void testInvalid() { diff --git a/core/src/test/java/org/elasticsearch/search/source/SourceFetchingIT.java b/core/src/test/java/org/elasticsearch/search/source/SourceFetchingIT.java index df147ce106f..a7df3bc6883 100644 --- a/core/src/test/java/org/elasticsearch/search/source/SourceFetchingIT.java +++ b/core/src/test/java/org/elasticsearch/search/source/SourceFetchingIT.java @@ -60,17 +60,17 @@ public class SourceFetchingIT extends ESIntegTestCase { response = client().prepareSearch("test").setFetchSource("field1", null).get(); assertThat(response.getHits().getAt(0).getSourceAsString(), notNullValue()); - assertThat(response.getHits().getAt(0).getSource().size(), equalTo(1)); - assertThat((String) response.getHits().getAt(0).getSource().get("field1"), equalTo("value")); + assertThat(response.getHits().getAt(0).getSourceAsMap().size(), equalTo(1)); + assertThat((String) response.getHits().getAt(0).getSourceAsMap().get("field1"), equalTo("value")); response = client().prepareSearch("test").setFetchSource("hello", null).get(); assertThat(response.getHits().getAt(0).getSourceAsString(), notNullValue()); - assertThat(response.getHits().getAt(0).getSource().size(), equalTo(0)); + assertThat(response.getHits().getAt(0).getSourceAsMap().size(), equalTo(0)); response = client().prepareSearch("test").setFetchSource(new String[]{"*"}, new String[]{"field2"}).get(); assertThat(response.getHits().getAt(0).getSourceAsString(), notNullValue()); - assertThat(response.getHits().getAt(0).getSource().size(), equalTo(1)); - assertThat((String) response.getHits().getAt(0).getSource().get("field1"), equalTo("value")); + assertThat(response.getHits().getAt(0).getSourceAsMap().size(), equalTo(1)); + assertThat((String) response.getHits().getAt(0).getSourceAsMap().get("field1"), equalTo("value")); } @@ -87,12 +87,12 @@ public class SourceFetchingIT extends ESIntegTestCase { SearchResponse response = client().prepareSearch("test").setFetchSource(new String[]{"*.notexisting","field"}, null).get(); assertThat(response.getHits().getAt(0).getSourceAsString(), notNullValue()); - assertThat(response.getHits().getAt(0).getSource().size(), equalTo(1)); - assertThat((String) response.getHits().getAt(0).getSource().get("field"), equalTo("value")); + assertThat(response.getHits().getAt(0).getSourceAsMap().size(), equalTo(1)); + assertThat((String) response.getHits().getAt(0).getSourceAsMap().get("field"), equalTo("value")); response = client().prepareSearch("test").setFetchSource(new String[]{"field.notexisting.*","field"}, null).get(); assertThat(response.getHits().getAt(0).getSourceAsString(), notNullValue()); - assertThat(response.getHits().getAt(0).getSource().size(), equalTo(1)); - assertThat((String) response.getHits().getAt(0).getSource().get("field"), equalTo("value")); + assertThat(response.getHits().getAt(0).getSourceAsMap().size(), equalTo(1)); + assertThat((String) response.getHits().getAt(0).getSourceAsMap().get("field"), equalTo("value")); } } diff --git a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java index a86b63c1a59..00b8fc8e932 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.search.suggest; -import com.carrotsearch.hppc.ObjectLongHashMap; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.lucene.analysis.TokenStreamToAutomaton; @@ -218,7 +217,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { assertThat(option.getText().toString(), equalTo("suggestion" + id)); assertSearchHit(option.getHit(), hasId("" + id)); assertSearchHit(option.getHit(), hasScore(((float) id))); - assertNotNull(option.getHit().source()); + assertNotNull(option.getHit().getSourceAsMap()); id--; } } @@ -253,7 +252,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { assertThat(option.getText().toString(), equalTo("suggestion" + id)); assertSearchHit(option.getHit(), hasId("" + id)); assertSearchHit(option.getHit(), hasScore(((float) id))); - assertNull(option.getHit().source()); + assertNull(option.getHit().getSourceAsMap()); id--; } } @@ -290,8 +289,8 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { assertThat(option.getText().toString(), equalTo("suggestion" + id)); assertSearchHit(option.getHit(), hasId("" + id)); assertSearchHit(option.getHit(), hasScore(((float) id))); - assertNotNull(option.getHit().source()); - Set sourceFields = option.getHit().sourceAsMap().keySet(); + assertNotNull(option.getHit().getSourceAsMap()); + Set sourceFields = option.getHit().getSourceAsMap().keySet(); assertThat(sourceFields, contains("a")); assertThat(sourceFields, not(contains("b"))); id--; @@ -973,7 +972,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { refresh(); assertSuggestions("b"); - assertThat(2L, equalTo(client().prepareSearch(INDEX).setSize(0).get().getHits().totalHits())); + assertThat(2L, equalTo(client().prepareSearch(INDEX).setSize(0).get().getHits().getTotalHits())); for (IndexShardSegments seg : client().admin().indices().prepareSegments().get().getIndices().get(INDEX)) { ShardSegments[] shards = seg.getShards(); for (ShardSegments shardSegments : shards) { diff --git a/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java b/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java index 66c3509c76d..c925e46cfa0 100644 --- a/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java +++ b/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java @@ -65,12 +65,12 @@ public class SimilarityIT extends ESIntegTestCase { .setRefreshPolicy(IMMEDIATE).execute().actionGet(); SearchResponse bm25SearchResponse = client().prepareSearch().setQuery(matchQuery("field1", "quick brown fox")).execute().actionGet(); - assertThat(bm25SearchResponse.getHits().totalHits(), equalTo(1L)); - float bm25Score = bm25SearchResponse.getHits().hits()[0].score(); + assertThat(bm25SearchResponse.getHits().getTotalHits(), equalTo(1L)); + float bm25Score = bm25SearchResponse.getHits().getHits()[0].getScore(); SearchResponse defaultSearchResponse = client().prepareSearch().setQuery(matchQuery("field2", "quick brown fox")).execute().actionGet(); - assertThat(defaultSearchResponse.getHits().totalHits(), equalTo(1L)); - float defaultScore = defaultSearchResponse.getHits().hits()[0].score(); + assertThat(defaultSearchResponse.getHits().getTotalHits(), equalTo(1L)); + float defaultScore = defaultSearchResponse.getHits().getHits()[0].getScore(); assertThat(bm25Score, not(equalTo(defaultScore))); } diff --git a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index c334ba6e7a9..8f6701977e4 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -346,7 +346,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> create repository"); logger.info("--> creating repository"); @@ -394,7 +394,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> creating repository"); Path repo = randomRepoPath(); @@ -462,7 +462,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest index("test-idx-some", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> shutdown one of the nodes"); internalCluster().stopRandomDataNode(); @@ -483,7 +483,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest index("test-idx-closed", "doc", Integer.toString(i), "foo", "bar" + i); } refresh("test-idx-closed", "test-idx-all"); // don't refresh test-idx-some it will take 30 sec until it times out... - assertThat(client().prepareSearch("test-idx-all").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch("test-idx-all").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); assertAcked(client().admin().indices().prepareClose("test-idx-closed")); logger.info("--> create an index that will have no allocated shards"); @@ -570,7 +570,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(6)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - assertThat(client().prepareSearch("test-idx-all").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch("test-idx-all").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> restore snapshot for the partial index"); cluster().wipeIndices("test-idx-some"); @@ -581,7 +581,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0), lessThan(6))); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), greaterThan(0)); - assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().totalHits(), allOf(greaterThan(0L), lessThan(100L))); + assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().getTotalHits(), allOf(greaterThan(0L), lessThan(100L))); logger.info("--> restore snapshot for the index that didn't have any shards snapshotted successfully"); cluster().wipeIndices("test-idx-none"); @@ -592,7 +592,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(0)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(6)); - assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().totalHits(), allOf(greaterThan(0L), lessThan(100L))); + assertThat(client().prepareSearch("test-idx-some").setSize(0).get().getHits().getTotalHits(), allOf(greaterThan(0L), lessThan(100L))); } public void testRestoreIndexWithShardsMissingInLocalGateway() throws Exception { @@ -620,7 +620,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client().prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> start snapshot"); assertThat(client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1").setIndices("test-idx").setWaitForCompletion(true).get().getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); @@ -642,7 +642,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest assertThat(client().admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap-1").setRestoreGlobalState(false).setWaitForCompletion(true).get().getRestoreInfo().successfulShards(), equalTo(6)); ensureGreen("test-idx"); - assertThat(client().prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client().prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); IntSet reusedShards = new IntHashSet(); for (RecoveryState recoveryState : client().admin().indices().prepareRecoveries("test-idx").get().shardRecoveryStates().get("test-idx")) { diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index aa229b0910c..b552daafb09 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -611,7 +611,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot without global state but with indices"); createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-no-global-state-with-index").setIndices("test-idx").setIncludeGlobalState(false).setWaitForCompletion(true).get(); @@ -645,7 +645,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); assertFalse(client().admin().cluster().prepareGetPipeline("barbaz").get().isFound()); assertNull(client().admin().cluster().prepareGetStoredScript(MockScriptEngine.NAME, "foobar").get().getSource()); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); } @@ -669,7 +669,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); try { @@ -718,7 +718,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get(); @@ -780,7 +780,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get(); @@ -802,7 +802,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); SearchResponse countResponse = client.prepareSearch("test-idx").setSize(0).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(100L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(100L)); logger.info("--> total number of simulated failures during restore: [{}]", getFailureCount("test-repo")); } @@ -821,7 +821,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get(); @@ -862,7 +862,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get(); @@ -906,7 +906,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); SearchResponse countResponse = client.prepareSearch("test-idx").setSize(0).get(); - assertThat(countResponse.getHits().totalHits(), equalTo(100L)); + assertThat(countResponse.getHits().getTotalHits(), equalTo(100L)); } @@ -958,7 +958,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas // Store number of files after each snapshot numberOfFiles[i] = numberOfFiles(repo); } - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(10L * numberOfSnapshots)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(10L * numberOfSnapshots)); int numberOfFilesBeforeDeletion = numberOfFiles(repo); logger.info("--> delete all snapshots except the first one and last one"); @@ -978,7 +978,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", lastSnapshot).setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(10L * numberOfSnapshots)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(10L * numberOfSnapshots)); logger.info("--> delete the last snapshot"); client.admin().cluster().prepareDeleteSnapshot("test-repo", lastSnapshot).get(); @@ -1203,8 +1203,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx-2", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-1", "test-idx-2").get(); @@ -1216,8 +1216,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx-1-copy").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2-copy").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1-copy").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2-copy").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> close just restored indices"); client.admin().indices().prepareClose("test-idx-1-copy", "test-idx-2-copy").get(); @@ -1227,8 +1227,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx-1-copy").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2-copy").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1-copy").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2-copy").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> close indices"); @@ -1318,7 +1318,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); // Pick one node and block it String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); @@ -1357,7 +1357,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); } public void testDeleteRepositoryWhileSnapshotting() throws Exception { @@ -1381,7 +1381,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); // Pick one node and block it String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); @@ -1439,7 +1439,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); } @TestLogging("_root:DEBUG") // this fails every now and then: https://github.com/elastic/elasticsearch/issues/18121 but without @@ -1484,7 +1484,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("readonly-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> list available shapshots"); GetSnapshotsResponse getSnapshotsResponse = client.admin().cluster().prepareGetSnapshots("readonly-repo").get(); @@ -1521,7 +1521,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get(); @@ -1534,7 +1534,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas logger.info("--> restore index"); RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); long snapshotPause = 0L; long restorePause = 0L; @@ -1577,7 +1577,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); // Pick one node and block it String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); @@ -1683,7 +1683,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> start relocations"); allowNodes("test-idx", internalCluster().numDataNodes()); @@ -2003,9 +2003,9 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx-3", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot allow partial {}", allowPartial); ListenableActionFuture future = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") @@ -2090,8 +2090,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx-2", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); assertThat(client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") @@ -2152,7 +2152,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index(indexName, "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> take snapshots"); final String snapshotName = "test-snap"; @@ -2697,7 +2697,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo((long) numDocs)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs)); logger.info("--> snapshot with potential I/O failures"); try { diff --git a/core/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/core/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index ccb392be1fc..cfaf9203fea 100644 --- a/core/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/core/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -237,13 +237,13 @@ public class SimpleVersioningIT extends ESIntegTestCase { // search with versioning for (int i = 0; i < 10; i++) { SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setVersion(true).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).version(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getVersion(), equalTo(2L)); } // search without versioning for (int i = 0; i < 10; i++) { SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).version(), equalTo(Versions.NOT_FOUND)); + assertThat(searchResponse.getHits().getAt(0).getVersion(), equalTo(Versions.NOT_FOUND)); } DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(2).execute().actionGet(); @@ -293,7 +293,7 @@ public class SimpleVersioningIT extends ESIntegTestCase { for (int i = 0; i < 10; i++) { SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setVersion(true).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).version(), equalTo(2L)); + assertThat(searchResponse.getHits().getAt(0).getVersion(), equalTo(2L)); } } diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java index 19a70238336..d123b2f307e 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java @@ -86,7 +86,7 @@ public class SearchTemplateIT extends ESSingleNodeTestCase { .setScriptParams(Collections.singletonMap("my_size", 1)) .get(); - assertThat(searchResponse.getResponse().getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getResponse().getHits().getHits().length, equalTo(1)); } /** @@ -104,7 +104,7 @@ public class SearchTemplateIT extends ESSingleNodeTestCase { SearchTemplateRequest request = RestSearchTemplateAction.parse(createParser(JsonXContent.jsonXContent, query)); request.setRequest(searchRequest); SearchTemplateResponse searchResponse = client().execute(SearchTemplateAction.INSTANCE, request).get(); - assertThat(searchResponse.getResponse().getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getResponse().getHits().getHits().length, equalTo(1)); } /** @@ -125,7 +125,7 @@ public class SearchTemplateIT extends ESSingleNodeTestCase { SearchTemplateRequest request = RestSearchTemplateAction.parse(createParser(JsonXContent.jsonXContent, templateString)); request.setRequest(searchRequest); SearchTemplateResponse searchResponse = client().execute(SearchTemplateAction.INSTANCE, request).get(); - assertThat(searchResponse.getResponse().getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getResponse().getHits().getHits().length, equalTo(1)); } /** @@ -146,7 +146,7 @@ public class SearchTemplateIT extends ESSingleNodeTestCase { SearchTemplateRequest request = RestSearchTemplateAction.parse(createParser(JsonXContent.jsonXContent, templateString)); request.setRequest(searchRequest); SearchTemplateResponse searchResponse = client().execute(SearchTemplateAction.INSTANCE, request).get(); - assertThat(searchResponse.getResponse().getHits().hits().length, equalTo(1)); + assertThat(searchResponse.getResponse().getHits().getHits().length, equalTo(1)); } public void testIndexedTemplateClient() throws Exception { diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorHighlightSubFetchPhase.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorHighlightSubFetchPhase.java index db0745a045e..97533d23ee4 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorHighlightSubFetchPhase.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorHighlightSubFetchPhase.java @@ -37,7 +37,7 @@ import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.highlight.HighlightPhase; import org.elasticsearch.search.fetch.subphase.highlight.Highlighter; import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight; -import org.elasticsearch.search.internal.InternalSearchHit; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SubSearchContext; @@ -62,7 +62,7 @@ public final class PercolatorHighlightSubFetchPhase extends HighlightPhase { } @Override - public void hitsExecute(SearchContext context, InternalSearchHit[] hits) { + public void hitsExecute(SearchContext context, SearchHit[] hits) { if (hitsExecutionNeeded(context) == false) { return; } @@ -81,7 +81,7 @@ public final class PercolatorHighlightSubFetchPhase extends HighlightPhase { SubSearchContext subSearchContext = createSubSearchContext(context, percolatorLeafReaderContext, percolateQuery.getDocumentSource()); - for (InternalSearchHit hit : hits) { + for (SearchHit hit : hits) { final Query query; try { LeafReaderContext ctx = ctxs.get(ReaderUtil.subIndex(hit.docId(), ctxs)); @@ -93,12 +93,12 @@ public final class PercolatorHighlightSubFetchPhase extends HighlightPhase { if (query != null) { subSearchContext.parsedQuery(new ParsedQuery(query)); hitContext.reset( - new InternalSearchHit(0, "unknown", new Text(percolateQuery.getDocumentType()), Collections.emptyMap()), + new SearchHit(0, "unknown", new Text(percolateQuery.getDocumentType()), Collections.emptyMap()), percolatorLeafReaderContext, 0, percolatorIndexSearcher ); hitContext.cache().clear(); super.hitExecute(subSearchContext, hitContext); - hit.highlightFields().putAll(hitContext.hit().getHighlightFields()); + hit.getHighlightFields().putAll(hitContext.hit().getHighlightFields()); } } } diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java index 07fc9aa2fe6..43d4e4388c4 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorQuerySearchIT.java @@ -383,13 +383,13 @@ public class PercolatorQuerySearchIT extends ESSingleNodeTestCase { .get(); assertHitCount(response, 4); assertThat(response.getHits().getAt(0).getId(), equalTo("1")); - assertThat(response.getHits().getAt(0).score(), equalTo(Float.NaN)); + assertThat(response.getHits().getAt(0).getScore(), equalTo(Float.NaN)); assertThat(response.getHits().getAt(1).getId(), equalTo("2")); - assertThat(response.getHits().getAt(1).score(), equalTo(Float.NaN)); + assertThat(response.getHits().getAt(1).getScore(), equalTo(Float.NaN)); assertThat(response.getHits().getAt(2).getId(), equalTo("3")); - assertThat(response.getHits().getAt(2).score(), equalTo(Float.NaN)); + assertThat(response.getHits().getAt(2).getScore(), equalTo(Float.NaN)); assertThat(response.getHits().getAt(3).getId(), equalTo("4")); - assertThat(response.getHits().getAt(3).score(), equalTo(Float.NaN)); + assertThat(response.getHits().getAt(3).getScore(), equalTo(Float.NaN)); } public void testPercolatorQueryWithHighlighting() throws Exception { @@ -524,8 +524,8 @@ public class PercolatorQuerySearchIT extends ESSingleNodeTestCase { .get(); assertHitCount(response, 1); assertThat(response.getHits().getAt(0).getId(), equalTo("1")); - assertThat(response.getHits().getAt(0).type(), equalTo("query_type")); - assertThat(response.getHits().getAt(0).index(), equalTo("test1")); + assertThat(response.getHits().getAt(0).getType(), equalTo("query_type")); + assertThat(response.getHits().getAt(0).getIndex(), equalTo("test1")); response = client().prepareSearch() .setQuery(new PercolateQueryBuilder("object_field." + queryFieldName, "doc_type", source, XContentType.JSON)) @@ -533,8 +533,8 @@ public class PercolatorQuerySearchIT extends ESSingleNodeTestCase { .get(); assertHitCount(response, 1); assertThat(response.getHits().getAt(0).getId(), equalTo("1")); - assertThat(response.getHits().getAt(0).type(), equalTo("query_type")); - assertThat(response.getHits().getAt(0).index(), equalTo("test2")); + assertThat(response.getHits().getAt(0).getType(), equalTo("query_type")); + assertThat(response.getHits().getAt(0).getIndex(), equalTo("test2")); // Unacceptable: MapperParsingException e = expectThrows(MapperParsingException.class, () -> { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java index d40aed4b898..4d920600a5d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/DeleteByQueryBasicTests.java @@ -148,7 +148,7 @@ public class DeleteByQueryBasicTests extends ReindexTestCase { String routing = String.valueOf(randomIntBetween(2, docs)); logger.info("--> counting documents with routing [{}]", routing); - long expected = client().prepareSearch().setSize(0).setRouting(routing).get().getHits().totalHits(); + long expected = client().prepareSearch().setSize(0).setRouting(routing).get().getHits().getTotalHits(); logger.info("--> delete all documents with routing [{}] with a delete-by-query", routing); DeleteByQueryRequestBuilder delete = deleteByQuery().source("test"); diff --git a/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java b/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java index f7fb7af055c..035eb2a26d6 100644 --- a/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java +++ b/modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLSnapshotRestoreTests.java @@ -68,7 +68,7 @@ public class URLSnapshotRestoreTests extends ESIntegTestCase { index("test-idx", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client @@ -112,7 +112,7 @@ public class URLSnapshotRestoreTests extends ESIntegTestCase { .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> list available shapshots"); GetSnapshotsResponse getSnapshotsResponse = client.admin().cluster().prepareGetSnapshots("url-repo").get(); diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSnapshotRestoreTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSnapshotRestoreTests.java index d1050e80adc..d149cc9d201 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSnapshotRestoreTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureSnapshotRestoreTests.java @@ -114,9 +114,9 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT index("test-idx-3", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") @@ -139,9 +139,9 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT client.prepareDelete("test-idx-3", "doc", Integer.toString(i)).get(); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); logger.info("--> close indices"); client.admin().indices().prepareClose("test-idx-1", "test-idx-2").get(); @@ -152,9 +152,9 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); // Test restore after index deletion logger.info("--> delete indices"); @@ -164,7 +164,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT .setIndices("test-idx-*", "-test-idx-2").get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false)); @@ -189,7 +189,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT logger.info("indexing first document"); index(indexName, typeName, Integer.toString(1), "foo", "bar " + Integer.toString(1)); refresh(); - assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().totalHits(), equalTo(1L)); + assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(), equalTo(1L)); logger.info("creating Azure repository with path [{}]", getRepositoryPath()); PutRepositoryResponse putRepositoryResponse = client.admin().cluster().preparePutRepository(repositoryName) @@ -213,7 +213,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT logger.info("indexing second document"); index(indexName, typeName, Integer.toString(2), "foo", "bar " + Integer.toString(2)); refresh(); - assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().totalHits(), equalTo(2L)); + assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(), equalTo(2L)); logger.info("creating snapshot [{}]", snapshot2Name); CreateSnapshotResponse createSnapshotResponse2 = client.admin().cluster().prepareCreateSnapshot(repositoryName, snapshot2Name) @@ -233,7 +233,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT .setWaitForCompletion(true).get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().totalHits(), equalTo(1L)); + assertThat(client.prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(), equalTo(1L)); } public void testMultipleRepositories() { @@ -263,8 +263,8 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT index("test-idx-2", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot 1"); CreateSnapshotResponse createSnapshotResponse1 = client.admin().cluster().prepareCreateSnapshot("test-repo1", "test-snap") @@ -293,7 +293,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT .setWaitForCompletion(true).setIndices("test-idx-1").get(); assertThat(restoreSnapshotResponse1.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false)); @@ -303,7 +303,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyIntegT .setWaitForCompletion(true).setIndices("test-idx-2").get(); assertThat(restoreSnapshotResponse2.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(true)); diff --git a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java index a6ed0858474..511aafdd29b 100644 --- a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java +++ b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.RepositoryException; -import org.elasticsearch.repositories.hdfs.HdfsPlugin; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -184,6 +183,6 @@ public class HdfsTests extends ESSingleNodeTestCase { } private long count(Client client, String index) { - return client.prepareSearch(index).setSize(0).get().getHits().totalHits(); + return client.prepareSearch(index).setSize(0).get().getHits().getTotalHits(); } } diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AbstractS3SnapshotRestoreTest.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AbstractS3SnapshotRestoreTest.java index d38c61f4a37..1bd3341cf37 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AbstractS3SnapshotRestoreTest.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AbstractS3SnapshotRestoreTest.java @@ -19,7 +19,6 @@ package org.elasticsearch.repositories.s3; -import com.amazonaws.Protocol; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.ObjectListing; @@ -108,9 +107,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase index("test-idx-3", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-*", "-test-idx-3").get(); @@ -130,9 +129,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase client.prepareDelete("test-idx-3", "doc", Integer.toString(i)).get(); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); logger.info("--> close indices"); client.admin().indices().prepareClose("test-idx-1", "test-idx-2").get(); @@ -142,9 +141,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); // Test restore after index deletion logger.info("--> delete indices"); @@ -153,7 +152,7 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-*", "-test-idx-2").execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false)); @@ -184,9 +183,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase index("test-idx-3", "doc", Integer.toString(i), "foo", "baz" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-*", "-test-idx-3").get(); @@ -217,9 +216,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase client.prepareDelete("test-idx-3", "doc", Integer.toString(i)).get(); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(50L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); logger.info("--> close indices"); client.admin().indices().prepareClose("test-idx-1", "test-idx-2").get(); @@ -229,9 +228,9 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().totalHits(), equalTo(100L)); - assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-2").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-3").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); // Test restore after index deletion logger.info("--> delete indices"); @@ -240,7 +239,7 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-*", "-test-idx-2").execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetaData().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetaData().hasIndex("test-idx-2"), equalTo(false)); @@ -403,7 +402,7 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase index("test-idx-1", "doc", Integer.toString(i), "foo", "bar" + i); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repository, "test-snap").setWaitForCompletion(true).setIndices("test-idx-*").get(); @@ -417,7 +416,7 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase client.prepareDelete("test-idx-1", "doc", Integer.toString(i)).get(); } refresh(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(50L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(50L)); logger.info("--> close indices"); client.admin().indices().prepareClose("test-idx-1").get(); @@ -427,7 +426,7 @@ public abstract class AbstractS3SnapshotRestoreTest extends AbstractAwsTestCase assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().totalHits(), equalTo(100L)); + assertThat(client.prepareSearch("test-idx-1").setSize(0).get().getHits().getTotalHits(), equalTo(100L)); } diff --git a/plugins/store-smb/src/test/java/org/elasticsearch/index/store/AbstractAzureFsTestCase.java b/plugins/store-smb/src/test/java/org/elasticsearch/index/store/AbstractAzureFsTestCase.java index 43b412f2e30..3033f6d4008 100644 --- a/plugins/store-smb/src/test/java/org/elasticsearch/index/store/AbstractAzureFsTestCase.java +++ b/plugins/store-smb/src/test/java/org/elasticsearch/index/store/AbstractAzureFsTestCase.java @@ -44,6 +44,6 @@ public abstract class AbstractAzureFsTestCase extends ESIntegTestCase { } refresh(); SearchResponse response = client().prepareSearch("test").get(); - assertThat(response.getHits().totalHits(), is(nbDocs)); + assertThat(response.getHits().getTotalHits(), is(nbDocs)); } } diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java index d61155ecd60..1bb5bb58eaf 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java @@ -83,7 +83,7 @@ public abstract class ESBlobStoreRepositoryIntegTestCase extends ESIntegTestCase logger.info("--> add random documents to {}", index); addRandomDocuments(index, randomIntBetween(10, 1000)); } else { - int docCount = (int) client().prepareSearch(index).setSize(0).get().getHits().totalHits(); + int docCount = (int) client().prepareSearch(index).setSize(0).get().getHits().getTotalHits(); int deleteCount = randomIntBetween(1, docCount); logger.info("--> delete {} random documents from {}", deleteCount, index); for (int i = 0; i < deleteCount; i++) { @@ -138,7 +138,7 @@ public abstract class ESBlobStoreRepositoryIntegTestCase extends ESIntegTestCase addRandomDocuments(indexName, docCount); } // Check number of documents in this iteration - docCounts[i] = (int) client().prepareSearch(indexName).setSize(0).get().getHits().totalHits(); + docCounts[i] = (int) client().prepareSearch(indexName).setSize(0).get().getHits().getTotalHits(); logger.info("--> create snapshot {}:{} with {} documents", repoName, snapshotName + "-" + i, docCounts[i]); assertSuccessfulSnapshot(client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName + "-" + i) .setWaitForCompletion(true).setIndices(indexName)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 92aafd4e8a3..2d30d6d73a5 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -857,7 +857,7 @@ public abstract class ESIntegTestCase extends ESTestCase { String failMsg = sb.toString(); for (SearchHit hit : searchResponse.getHits().getHits()) { sb.append("\n-> _index: [").append(hit.getIndex()).append("] type [").append(hit.getType()) - .append("] id [").append(hit.id()).append("]"); + .append("] id [").append(hit.getId()).append("]"); } logger.warn("{}", sb); fail(failMsg); @@ -1014,7 +1014,7 @@ public abstract class ESIntegTestCase extends ESTestCase { } if (lastKnownCount.get() >= numDocs) { try { - long count = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().totalHits(); + long count = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet().getHits().getTotalHits(); if (count == lastKnownCount.get()) { // no progress - try to refresh for the next time client().admin().indices().prepareRefresh().get(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index 6280de426ce..4e57503df4b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -222,7 +222,7 @@ public class ElasticsearchAssertions { Set idsSet = new HashSet<>(Arrays.asList(ids)); for (SearchHit hit : searchResponse.getHits()) { - assertThat("id [" + hit.getId() + "] was found in search results but wasn't expected (type [" + hit.getType() + "], index [" + hit.index() + "])" + assertThat("id [" + hit.getId() + "] was found in search results but wasn't expected (type [" + hit.getType() + "], index [" + hit.getIndex() + "])" + shardStatus, idsSet.remove(hit.getId()), equalTo(true)); } @@ -244,17 +244,17 @@ public class ElasticsearchAssertions { public static void assertOrderedSearchHits(SearchResponse searchResponse, String... ids) { String shardStatus = formatShardStatus(searchResponse); - assertThat("Expected different hit count. " + shardStatus, searchResponse.getHits().hits().length, equalTo(ids.length)); + assertThat("Expected different hit count. " + shardStatus, searchResponse.getHits().getHits().length, equalTo(ids.length)); for (int i = 0; i < ids.length; i++) { - SearchHit hit = searchResponse.getHits().hits()[i]; + SearchHit hit = searchResponse.getHits().getHits()[i]; assertThat("Expected id: " + ids[i] + " at position " + i + " but wasn't." + shardStatus, hit.getId(), equalTo(ids[i])); } assertVersionSerializable(searchResponse); } public static void assertHitCount(SearchResponse countResponse, long expectedHitCount) { - if (countResponse.getHits().totalHits() != expectedHitCount) { - fail("Count is " + countResponse.getHits().totalHits() + " but " + expectedHitCount + " was expected. " + formatShardStatus(countResponse)); + if (countResponse.getHits().getTotalHits() != expectedHitCount) { + fail("Count is " + countResponse.getHits().getTotalHits() + " but " + expectedHitCount + " was expected. " + formatShardStatus(countResponse)); } assertVersionSerializable(countResponse); } @@ -287,7 +287,7 @@ public class ElasticsearchAssertions { public static void assertSearchHit(SearchResponse searchResponse, int number, Matcher matcher) { assertThat(number, greaterThan(0)); assertThat("SearchHit number must be greater than 0", number, greaterThan(0)); - assertThat(searchResponse.getHits().totalHits(), greaterThanOrEqualTo((long) number)); + assertThat(searchResponse.getHits().getTotalHits(), greaterThanOrEqualTo((long) number)); assertSearchHit(searchResponse.getHits().getAt(number - 1), matcher); assertVersionSerializable(searchResponse); } @@ -375,21 +375,21 @@ public class ElasticsearchAssertions { private static void assertHighlight(SearchResponse resp, int hit, String field, int fragment, Matcher fragmentsMatcher, Matcher matcher) { assertNoFailures(resp); - assertThat("not enough hits", resp.getHits().hits().length, greaterThan(hit)); - assertHighlight(resp.getHits().hits()[hit], field, fragment, fragmentsMatcher, matcher); + assertThat("not enough hits", resp.getHits().getHits().length, greaterThan(hit)); + assertHighlight(resp.getHits().getHits()[hit], field, fragment, fragmentsMatcher, matcher); assertVersionSerializable(resp); } private static void assertHighlight(SearchHit hit, String field, int fragment, Matcher fragmentsMatcher, Matcher matcher) { assertThat(hit.getHighlightFields(), hasKey(field)); assertThat(hit.getHighlightFields().get(field).fragments().length, fragmentsMatcher); - assertThat(hit.highlightFields().get(field).fragments()[fragment].string(), matcher); + assertThat(hit.getHighlightFields().get(field).fragments()[fragment].string(), matcher); } public static void assertNotHighlighted(SearchResponse resp, int hit, String field) { assertNoFailures(resp); - assertThat("not enough hits", resp.getHits().hits().length, greaterThan(hit)); - assertThat(resp.getHits().hits()[hit].getHighlightFields(), not(hasKey(field))); + assertThat("not enough hits", resp.getHits().getHits().length, greaterThan(hit)); + assertThat(resp.getHits().getHits()[hit].getHighlightFields(), not(hasKey(field))); } public static void assertSuggestionSize(Suggest searchSuggest, int entry, int size, String key) {