diff --git a/CHANGELOG.md b/CHANGELOG.md index bd9ec337e9449..3e8f6a5a402c7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -36,6 +36,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Star-Tree] Add star-tree search related stats ([#18707](https://github.com/opensearch-project/OpenSearch/pull/18707)) - Add support for plugins to profile information ([#18656](https://github.com/opensearch-project/OpenSearch/pull/18656)) - Add support for Combined Fields query ([#18724](https://github.com/opensearch-project/OpenSearch/pull/18724)) +- Terminate eligible non-scoring boolean queries early for performance ([#18842](https://github.com/opensearch-project/OpenSearch/pull/18842)) - Added approximation support for range queries with now in date field ([#18511](https://github.com/opensearch-project/OpenSearch/pull/18511)) ### Changed diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/BooleanQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/BooleanQueryIT.java index b185b8b0131d1..72c480990834e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/BooleanQueryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/BooleanQueryIT.java @@ -10,12 +10,21 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.search.SearchHit; +import org.opensearch.search.sort.SortOrder; +import org.opensearch.search.suggest.SuggestBuilder; +import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; +import org.opensearch.transport.client.Client; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -26,8 +35,14 @@ import static org.opensearch.index.query.QueryBuilders.termQuery; import static org.opensearch.index.query.QueryBuilders.termsQuery; import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; +import static org.opensearch.search.aggregations.AggregationBuilders.cardinality; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertOrderedSearchHits; +import static org.apache.lucene.search.TotalHits.Relation.EQUAL_TO; +import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1) public class BooleanQueryIT extends ParameterizedStaticSettingsOpenSearchIntegTestCase { public BooleanQueryIT(Settings staticSettings) { @@ -69,10 +84,7 @@ public void testMustNotRangeRewrite() throws Exception { .setSource(intField, i, termField1, termValue1, termField2, termValue2) .get(); } - ensureGreen(); - waitForRelocation(); - forceMerge(); - refresh(); + afterIndexing(); int lt = 80; int gte = 20; @@ -118,10 +130,7 @@ public void testMustNotDateRangeRewrite() throws Exception { for (int day = minDay; day <= maxDay; day++) { client().prepareIndex("test").setSource(dateField, getDate(day, 0)).get(); } - ensureGreen(); - waitForRelocation(); - forceMerge(); - refresh(); + afterIndexing(); int minExcludedDay = 15; int maxExcludedDay = 25; @@ -147,10 +156,7 @@ public void testMustNotDateRangeWithFormatAndTimeZoneRewrite() throws Exception for (int hour = 0; hour < 24; hour++) { client().prepareIndex("test").setSource(dateField, getDate(1, hour)).get(); } - ensureGreen(); - waitForRelocation(); - forceMerge(); - refresh(); + afterIndexing(); int zoneOffset = 3; assertHitCount( @@ -185,10 +191,7 @@ public void testMustNotRangeRewriteWithMissingValues() throws Exception { client().prepareIndex("test").setId(Integer.toString(i)).setSource(termField, termValue).get(); } } - ensureGreen(); - waitForRelocation(); - forceMerge(); - refresh(); + afterIndexing(); // Search excluding range 30 to 50 @@ -217,10 +220,7 @@ public void testMustNotRangeRewriteWithMoreThanOneValue() throws Exception { client().prepareIndex("test").setId(Integer.toString(i)).setSource(intField, i).get(); } } - ensureGreen(); - waitForRelocation(); - forceMerge(); - refresh(); + afterIndexing(); // Range queries will match if ANY of the doc's values are within the range. // So if we exclude the range 0 to 20, we shouldn't see any of those documents returned, @@ -234,6 +234,145 @@ public void testMustNotRangeRewriteWithMoreThanOneValue() throws Exception { assertHitCount(client().prepareSearch().setQuery(matchAllQuery()).get(), numDocs); } + public void testFilterEarlyTermination() throws Exception { + int numDocs = 6_000; + String intField = "int_field"; + String textField1 = "text_field_1"; + List textField1Values = List.of("even", "odd"); + String textField2 = "text_field_2"; + List textField2Values = List.of("a", "b", "c"); + Client client = client(); + String indexName = "test"; + // Set trackTotalHitsUpTo to 500 rather than default 10k, so we have to index fewer docs + int trackTotalHitsUpTo = 500; + + // Enforce 1 shard per node, so that no shard has < trackTotalHitsUpTo matching docs and cannot actually terminate early + assertAcked( + client.admin() + .indices() + .prepareCreate(indexName) + .setMapping(intField, "type=integer", textField1, "type=text", textField2, "type=text") + .setSettings( + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + ) + .get() + ); + + for (int i = 0; i < numDocs; i++) { + String textValue1 = textField1Values.get(i % 2); + String textValue2 = textField2Values.get(i % 3); + client.prepareIndex(indexName) + .setId(Integer.toString(i)) + .setSource(intField, i, textField1, textValue1, textField2, textValue2) + .get(); + } + afterIndexing(); + int lte = 3000; + + // A query with only filter or must_not clauses should be able to terminate early + SearchResponse response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .get(); + assertTrue(response.isTerminatedEarly()); + // Note: queries that have finished early with terminate_after will return "eq" for hit relation + assertHitCount(response, trackTotalHitsUpTo, EQUAL_TO); + + // Force the same query to not terminate early by setting terminateAfter to some non-default high value, so we can check the hits + // are identical + SearchResponse originalQueryResponse = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .setTerminateAfter(1_000_000) + .get(); + assertFalse(originalQueryResponse.isTerminatedEarly()); // Returns false not null when TA was set but not reached + assertHitCount(originalQueryResponse, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + List terminatedEarlyIds = new ArrayList<>(); + for (Iterator it = response.getHits().iterator(); it.hasNext();) { + SearchHit terminatedEarlyHit = it.next(); + terminatedEarlyIds.add(terminatedEarlyHit.getId()); + } + assertOrderedSearchHits(originalQueryResponse, terminatedEarlyIds.toArray(new String[0])); + + // Queries with other clauses should not terminate early + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).must(termQuery(textField2, "a"))) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().must(rangeQuery(intField).lte(lte)).should(termQuery(textField1, "odd"))) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + // Queries with aggregations shouldn't terminate early + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .addAggregation(cardinality("cardinality").field(intField)) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + // Queries with sorting shouldn't terminate early + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .addSort(intField, SortOrder.DESC) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + // Queries with pagination shouldn't terminate early + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .setFrom(10) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + response = client().prepareSearch() + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .searchAfter(new Object[] { 0 }) + .addSort(intField, SortOrder.DESC) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + // Scroll queries shouldn't terminate early + response = client().prepareSearch() + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .setScroll(TimeValue.timeValueSeconds(30)) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, lte / 2); + + response = client().prepareSearchScroll(response.getScrollId()).setScroll(TimeValue.timeValueSeconds(30)).get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, lte / 2); + clearScroll(response.getScrollId()); + + // Suggest queries shouldn't terminate early + response = client().prepareSearch() + .suggest(new SuggestBuilder()) + .setTrackTotalHitsUpTo(trackTotalHitsUpTo) + .setQuery(boolQuery().mustNot(termQuery(textField1, "even")).filter(rangeQuery(intField).lte(lte))) + .get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + + // Search responses missing a query shouldn't terminate early or throw an error + response = client().prepareSearch().setTrackTotalHitsUpTo(trackTotalHitsUpTo).get(); + assertNull(response.isTerminatedEarly()); + assertHitCount(response, trackTotalHitsUpTo, GREATER_THAN_OR_EQUAL_TO); + } + public void testMustNotNumericMatchOrTermQueryRewrite() throws Exception { Map statusToDocCountMap = Map.of(200, 1000, 404, 30, 500, 1, 400, 1293); String statusField = "status"; @@ -313,4 +452,11 @@ private String padZeros(int value, int length) { private String getDate(int day, int hour) { return "2016-01-" + padZeros(day, 2) + "T" + padZeros(hour, 2) + ":00:00.000"; } + + private void afterIndexing() { + ensureGreen(); + waitForRelocation(); + forceMerge(); + refresh(); + } } diff --git a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java index 0dd4c3344af1e..597ec9ca6b353 100644 --- a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java @@ -1207,4 +1207,34 @@ public boolean evaluateKeywordIndexOrDocValuesEnabled() { } return false; } + + /** + * Checks if early termination can be enabled for this search, and enables it if so. + * @return whether early termination was applied + */ + boolean tryEnablingEarlyTermination() { + // This method should only be called after queries are rewritten and parsed, and terminateAfter and size have already been set. + assert size != -1 : "Cannot call `tryEnablingEarlyTermination` until after `size` has been set"; + assert from != -1 : "Cannot call `tryEnablingEarlyTermination` until after `from` has been set"; + if (query == null) return false; + + if (terminateAfter != DEFAULT_TERMINATE_AFTER) return false; + if (!(query instanceof BooleanQuery bq)) return false; + + if (aggregations() != null) return false; + if (from > 0 || searchAfter != null) return false; + if (sort != null) return false; + if (sliceBuilder != null || scrollContext() != null) return false; + if (suggest != null) return false; + + // We can only set terminateAfter to trackTotalHitsUpTo if we only have filter and must_not clauses + if (bq.getClauses(Occur.MUST).isEmpty() && bq.getClauses(Occur.SHOULD).isEmpty()) { + terminateAfter = Math.max(size, trackTotalHitsUpTo); + return true; + // TODO: Disabling concurrent segment search can speed this up even further for default trackTotalHitsUpTo of 10k, + // but at this point the CSS logic already assumes it can't be changed. We can revisit this in future. + } + // TODO: In future we can do the same for any constant-score query + return false; + } } diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index 6baf3b514198e..88659becfb3a6 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -1142,6 +1142,8 @@ final SearchContext createContext( // pre process queryPhase.preProcess(context); + + context.tryEnablingEarlyTermination(); } catch (Exception e) { context.close(); throw e; diff --git a/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java index 55b30d5068daa..1f46c8e005852 100644 --- a/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java @@ -36,6 +36,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCachingPolicy; @@ -67,6 +68,7 @@ import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.NumberFieldMapper; import org.opensearch.index.query.AbstractQueryBuilder; import org.opensearch.index.query.BoolQueryBuilder; import org.opensearch.index.query.ParsedQuery; @@ -106,6 +108,10 @@ import java.util.function.Supplier; import static org.opensearch.index.IndexSettings.INDEX_SEARCH_THROTTLED; +import static org.opensearch.index.query.QueryBuilders.boolQuery; +import static org.opensearch.index.query.QueryBuilders.rangeQuery; +import static org.opensearch.search.internal.SearchContext.DEFAULT_TERMINATE_AFTER; +import static org.opensearch.search.internal.SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.any; @@ -1170,6 +1176,160 @@ public Optional create(IndexSettings indexSettin } } + public void testEarlyTermination() throws Exception { + ShardSearchRequest shardSearchRequest = mock(ShardSearchRequest.class); + when(shardSearchRequest.searchType()).thenReturn(SearchType.DEFAULT); + ShardId shardId = new ShardId("index", UUID.randomUUID().toString(), 1); + when(shardSearchRequest.shardId()).thenReturn(shardId); + + ThreadPool threadPool = new TestThreadPool(this.getClass().getName()); + IndexShard indexShard = mock(IndexShard.class); + QueryCachingPolicy queryCachingPolicy = mock(QueryCachingPolicy.class); + when(indexShard.getQueryCachingPolicy()).thenReturn(queryCachingPolicy); + when(indexShard.getThreadPool()).thenReturn(threadPool); + + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) + .build(); + + IndexService indexService = mock(IndexService.class); + QueryShardContext queryShardContext = mock(QueryShardContext.class); + when(indexService.newQueryShardContext(eq(shardId.id()), any(), any(), nullable(String.class), anyBoolean(), anyBoolean())) + .thenReturn(queryShardContext); + MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType("value", NumberFieldMapper.NumberType.INTEGER); + when(queryShardContext.fieldMapper(anyString())).thenReturn(fieldType); + + IndexMetadata indexMetadata = IndexMetadata.builder("index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + when(indexService.getIndexSettings()).thenReturn(indexSettings); + when(indexShard.indexSettings()).thenReturn(indexSettings); + + BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + + try (Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir)) { + + final Supplier searcherSupplier = () -> new Engine.SearcherSupplier(Function.identity()) { + @Override + protected void doClose() {} + + @Override + protected Engine.Searcher acquireSearcherInternal(String source) { + try { + IndexReader reader = w.getReader(); + return new Engine.Searcher( + "test", + reader, + IndexSearcher.getDefaultSimilarity(), + IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), + reader + ); + } catch (IOException exc) { + throw new AssertionError(exc); + } + } + }; + + SearchShardTarget target = new SearchShardTarget("node", shardId, null, OriginalIndices.NONE); + ReaderContext readerContext = new ReaderContext( + newContextId(), + indexService, + indexShard, + searcherSupplier.get(), + randomNonNegativeLong(), + false + ); + + final ClusterService clusterService = mock(ClusterService.class); + final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterSettings.registerSetting(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING); + clusterSettings.registerSetting(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_MODE); + clusterSettings.applySettings( + Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_MODE.getKey(), "auto").build() + ); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + when(clusterService.getSettings()).thenReturn(settings); + + DefaultSearchContext context = new DefaultSearchContext( + readerContext, + shardSearchRequest, + target, + clusterService, + bigArrays, + null, + null, + null, + false, + Version.CURRENT, + false, + executor, + null, + Collections.emptyList() + ); + + // Test we can't run this method until size and from are set + assertThrows(AssertionError.class, context::tryEnablingEarlyTermination); + context.size(10); + assertThrows(AssertionError.class, context::tryEnablingEarlyTermination); + context.from(0); + // If no query is set yet, should return false. + assertFalse(context.tryEnablingEarlyTermination()); + // This should return true with a BooleanQuery with only filter/must_not clauses. + Query query = boolQuery().mustNot(rangeQuery("dummyField").gte(3)) + .filter(rangeQuery("dummyField2").lte(1)) + .toQuery(queryShardContext); + context.parsedQuery(new ParsedQuery(query)); + assertTrue(context.tryEnablingEarlyTermination()); + assertEquals(DEFAULT_TRACK_TOTAL_HITS_UP_TO, context.terminateAfter()); + context.terminateAfter(DEFAULT_TERMINATE_AFTER); + + // A BooleanQuery with other clause types should return false + Query mustQuery = boolQuery().must(rangeQuery("dummyField2").lte(1)).toQuery(queryShardContext); + context.parsedQuery(new ParsedQuery(mustQuery)); + assertFalse(context.tryEnablingEarlyTermination()); + context.parsedQuery(new ParsedQuery(query)); // Reset to constant-scoring boolean query + + // Scroll queries should return false + context.sliceBuilder(mock(SliceBuilder.class)); + assertFalse(context.tryEnablingEarlyTermination()); + context.sliceBuilder(null); // Reset + + // Sort queries should return false + DocValueFormat docValueFormat = mock(DocValueFormat.class); + context.sort(new SortAndFormats(new Sort(), new DocValueFormat[] { docValueFormat })); + assertFalse(context.tryEnablingEarlyTermination()); + context.sort(null); // Reset + + // From != 0 should return false + context.from(10); + assertFalse(context.tryEnablingEarlyTermination()); + context.from(0); // Reset + + // Agg queries should return false + SearchContextAggregations mockAggregations = mock(SearchContextAggregations.class); + when(mockAggregations.factories()).thenReturn(mock(AggregatorFactories.class)); + when(mockAggregations.factories().allFactoriesSupportConcurrentSearch()).thenReturn(false); + when(mockAggregations.multiBucketConsumer()).thenReturn(mock(MultiBucketConsumerService.MultiBucketConsumer.class)); + context.aggregations(mockAggregations); + assertFalse(context.tryEnablingEarlyTermination()); + context.aggregations(null); // Reset + + // If terminateAfter is already set to something other than its default value, it should return false + assertEquals(0, context.terminateAfter()); + context.terminateAfter(10_000); + assertFalse(context.tryEnablingEarlyTermination()); + context.terminateAfter(DEFAULT_TERMINATE_AFTER); // Reset + + // Non boolean queries should return false + context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); + assertFalse(context.tryEnablingEarlyTermination()); + } + // shutdown the threadpool + threadPool.shutdown(); + } + private ShardSearchContextId newContextId() { return new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()); } diff --git a/test/framework/src/main/java/org/opensearch/test/hamcrest/OpenSearchAssertions.java b/test/framework/src/main/java/org/opensearch/test/hamcrest/OpenSearchAssertions.java index cd7669e52cdf8..4d82a69e356a5 100644 --- a/test/framework/src/main/java/org/opensearch/test/hamcrest/OpenSearchAssertions.java +++ b/test/framework/src/main/java/org/opensearch/test/hamcrest/OpenSearchAssertions.java @@ -322,6 +322,23 @@ public static void assertHitCount(SearchResponse countResponse, long minHitCount } } + public static void assertHitCount(SearchResponse countResponse, long expectedHitCount, TotalHits.Relation relation) { + final TotalHits totalHits = countResponse.getHits().getTotalHits(); + if (totalHits.value() != expectedHitCount) { + fail("Count is " + totalHits + " but " + expectedHitCount + " was expected. " + formatShardStatus(countResponse)); + } + if (totalHits.relation() != relation) { + fail( + "Count relation is " + + totalHits.relation().toString() + + " but " + + relation.toString() + + " was expected. " + + formatShardStatus(countResponse) + ); + } + } + public static void assertExists(GetResponse response) { String message = String.format(Locale.ROOT, "Expected %s/%s to exist, but does not", response.getIndex(), response.getId()); assertThat(message, response.isExists(), is(true));