Search: Remove the count search type.

This commit brings the benefits of the `count` search type to search requests
that have a `size` of 0:
 - a single round-trip to shards (no fetch phase)
 - ability to use the query cache

Since `count` now provides no benefits over `query_then_fetch`, it has been
deprecated.

Close #7630
This commit is contained in:
Adrien Grand 2015-01-14 11:19:32 +01:00
parent 171e415a47
commit a608db122d
47 changed files with 246 additions and 103 deletions

View File

@ -16,7 +16,7 @@ results from older indices will be served directly from the cache.
==================================
For now, the query cache will only cache the results of search requests
where <<count,`?search_type=count`>>, so it will not cache `hits`,
where `size=0`, so it will not cache `hits`,
but it will cache `hits.total`, <<search-aggregations,aggregations>>, and
<<search-suggesters,suggestions>>.
@ -80,8 +80,9 @@ caching on a *per-query* basis. If set, it overrides the index-level setting:
[source,json]
-----------------------------
curl 'localhost:9200/my_index/_search?search_type=count&query_cache=true' -d'
curl 'localhost:9200/my_index/_search?query_cache=true' -d'
{
"size": 0,
"aggs": {
"popular_colors": {
"terms": {

View File

@ -297,3 +297,8 @@ in their place.
The thrift and memcached transport plugins are no longer supported. Instead, use
either the HTTP transport (enabled by default) or the node or transport Java client.
=== `search_type=count` deprecation
The `count` search type has been deprecated. All benefits from this search type can
now be achieved by using the `query_then_fetch` search type (which is the
default) and setting `size` to `0`.

View File

@ -130,11 +130,12 @@ See <<index-modules-shard-query-cache>> for more details.
=== Returning only aggregation results
There are many occasions when aggregations are required but search hits are not. For these cases the hits can be ignored by
adding `search_type=count` to the request URL parameters. For example:
setting `size=0`. For example:
[source,js]
--------------------------------------------------
$ curl -XGET 'http://localhost:9200/twitter/tweet/_search?search_type=count' -d '{
$ curl -XGET 'http://localhost:9200/twitter/tweet/_search' -d '{
"size": 0,
"aggregations": {
"my_agg": {
"terms": {
@ -146,8 +147,7 @@ $ curl -XGET 'http://localhost:9200/twitter/tweet/_search?search_type=count' -d
'
--------------------------------------------------
Setting `search_type` to `count` avoids executing the fetch phase of the search making the request more efficient. See
<<search-request-search-type>> for more information on the `search_type` parameter.
Setting `size` to `0` avoids executing the fetch phase of the search making the request more efficient.
[float]
=== Metadata

View File

@ -26,13 +26,13 @@ the `query`, `aggregations`, `from`, `size`, and so on). Here is an example:
$ cat requests
{"index" : "test"}
{"query" : {"match_all" : {}}, "from" : 0, "size" : 10}
{"index" : "test", "search_type" : "count"}
{"index" : "test", "search_type" : "dfs_query_then_fetch"}
{"query" : {"match_all" : {}}}
{}
{"query" : {"match_all" : {}}}
{"query" : {"match_all" : {}}}
{"search_type" : "count"}
{"search_type" : "dfs_query_then_fetch"}
{"query" : {"match_all" : {}}}
$ curl -XGET localhost:9200/_msearch --data-binary @requests; echo

View File

@ -71,8 +71,9 @@ And here is a sample response:
`query_cache`::
Set to `true` or `false` to enable or disable the caching
of search results for requests where `?search_type=count`, ie
aggregations and suggestions. See <<index-modules-shard-query-cache>>.
of search results for requests where `size` is 0, ie
aggregations and suggestions (no top hits returned).
See <<index-modules-shard-query-cache>>.
`terminate_after`::

View File

@ -65,6 +65,8 @@ scoring.
[[count]]
==== Count
deprecated[2.0.0, `count` does not provide any benefits over `query_then_fetch` with a `size` of `0`]
Parameter value: *count*.
A special search type that returns the count that matched the search

View File

@ -141,14 +141,15 @@ level override the suggest text on the global level.
In the below example we request suggestions for the following suggest
text: `devloping distibutd saerch engies` on the `title` field with a
maximum of 3 suggestions per term inside the suggest text. Note that in
this example we use the `count` search type. This isn't required, but a
this example we set `size` to `0`. This isn't required, but a
nice optimization. The suggestions are gather in the `query` phase and
in the case that we only care about suggestions (so no hits) we don't
need to execute the `fetch` phase.
[source,js]
--------------------------------------------------
curl -s -XPOST 'localhost:9200/_search?search_type=count' -d '{
curl -s -XPOST 'localhost:9200/_search' -d '{
"size": 0,
"suggest" : {
"my-title-suggestions-1" : {
"text" : "devloping distibutd saerch engies",

View File

@ -94,7 +94,7 @@ Defaults to no terminate_after.
|`search_type` |The type of the search operation to perform. Can be
`dfs_query_then_fetch`, `dfs_query_and_fetch`, `query_then_fetch`,
`query_and_fetch`, `count`, `scan`. Defaults to `query_then_fetch`. See
`query_and_fetch`, `scan` or `count` deprecated[2.0,Replaced by `size: 0`]. Defaults to `query_then_fetch`. See
<<search-request-search-type,_Search Type_>> for
more details on the different types of search that can be performed.

View File

@ -89,7 +89,7 @@
},
"search_type": {
"type" : "string",
"description" : "Specific search type (eg. `dfs_then_fetch`, `count`, etc)"
"description" : "Specific search type (eg. `dfs_then_fetch`, `scan`, etc)"
},
"search_types": {
"type" : "list",

View File

@ -33,7 +33,7 @@
- index: test_2
- query:
match_all: {}
- search_type: count
- search_type: query_then_fetch
index: test_1
- query:
match: {foo: bar}

View File

@ -0,0 +1,33 @@
---
"search_type=count (deprecated) support":
- do:
indices.create:
index: test
- do:
index:
index: test
type: test
id: 1
body: { foo: bar }
- do:
index:
index: test
type: test
id: 2
body: { foo: bar }
- do:
indices.refresh:
index: [test]
- do:
search:
index: test
search_type: count
body:
query:
match:
foo: bar
- match: {hits.total: 2}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.search;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.ParseField;
/**
* Search type represent the manner at which the search operation is executed.
@ -57,7 +58,9 @@ public enum SearchType {
SCAN((byte) 4),
/**
* Only counts the results, will still execute aggregations and the like.
* @deprecated does not any improvements compared to {@link #QUERY_THEN_FETCH} with a `size` of {@code 0}
*/
@Deprecated
COUNT((byte) 5);
/**
@ -65,6 +68,8 @@ public enum SearchType {
*/
public static final SearchType DEFAULT = QUERY_THEN_FETCH;
private static final ParseField COUNT_VALUE = new ParseField("count").withAllDeprecated("query_then_fetch");
private byte id;
SearchType(byte id) {
@ -118,7 +123,7 @@ public enum SearchType {
return SearchType.QUERY_AND_FETCH;
} else if ("scan".equals(searchType)) {
return SearchType.SCAN;
} else if ("count".equals(searchType)) {
} else if (COUNT_VALUE.match(searchType)) {
return SearchType.COUNT;
} else {
throw new ElasticsearchIllegalArgumentException("No search type for [" + searchType + "]");

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.search;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.type.*;
import org.elasticsearch.action.support.ActionFilters;
@ -59,7 +60,8 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
TransportSearchDfsQueryAndFetchAction dfsQueryAndFetchAction,
TransportSearchQueryAndFetchAction queryAndFetchAction,
TransportSearchScanAction scanAction,
TransportSearchCountAction countAction, ActionFilters actionFilters) {
TransportSearchCountAction countAction,
ActionFilters actionFilters) {
super(settings, SearchAction.NAME, threadPool, transportService, actionFilters);
this.clusterService = clusterService;
this.dfsQueryThenFetchAction = dfsQueryThenFetchAction;
@ -68,10 +70,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
this.queryAndFetchAction = queryAndFetchAction;
this.scanAction = scanAction;
this.countAction = countAction;
this.optimizeSingleShard = this.settings.getAsBoolean("action.search.optimize_single_shard", true);
}
@Override
@ -106,6 +105,8 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
scanAction.execute(searchRequest, listener);
} else if (searchRequest.searchType() == SearchType.COUNT) {
countAction.execute(searchRequest, listener);
} else {
throw new ElasticsearchIllegalStateException("Unknown search type: [" + searchRequest.searchType() + "]");
}
}

View File

@ -20,7 +20,9 @@
package org.elasticsearch.action.search.type;
import com.carrotsearch.hppc.IntArrayList;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.NoShardAvailableActionException;
@ -325,7 +327,9 @@ public abstract class TransportSearchTypeAction extends TransportAction<SearchRe
// we only release search context that we did not fetch from if we are not scrolling
if (request.scroll() == null) {
for (AtomicArray.Entry<? extends QuerySearchResultProvider> entry : queryResults.asList()) {
if (docIdsToLoad.get(entry.index) == null) {
final TopDocs topDocs = entry.value.queryResult().queryResult().topDocs();
if (topDocs != null && topDocs.scoreDocs.length > 0 // the shard had matches
&& docIdsToLoad.get(entry.index) == null) { // but none of them made it to the global top docs
try {
DiscoveryNode node = nodes.get(entry.value.queryResult().shardTarget().nodeId());
if (node != null) { // should not happen (==null) but safeguard anyhow

View File

@ -423,8 +423,7 @@ public class Lucene {
return new ScoreDoc(in.readVInt(), in.readFloat());
}
public static void writeTopDocs(StreamOutput out, TopDocs topDocs, int from) throws IOException {
from = Math.min(from, topDocs.scoreDocs.length);
public static void writeTopDocs(StreamOutput out, TopDocs topDocs) throws IOException {
if (topDocs instanceof TopFieldDocs) {
out.writeBoolean(true);
TopFieldDocs topFieldDocs = (TopFieldDocs) topDocs;
@ -448,9 +447,8 @@ public class Lucene {
out.writeBoolean(sortField.getReverse());
}
out.writeVInt(topDocs.scoreDocs.length - from);
for (int i = from; i < topFieldDocs.scoreDocs.length; ++i) {
ScoreDoc doc = topFieldDocs.scoreDocs[i];
out.writeVInt(topDocs.scoreDocs.length);
for (ScoreDoc doc : topFieldDocs.scoreDocs) {
writeFieldDoc(out, (FieldDoc) doc);
}
} else {
@ -458,9 +456,8 @@ public class Lucene {
out.writeVInt(topDocs.totalHits);
out.writeFloat(topDocs.getMaxScore());
out.writeVInt(topDocs.scoreDocs.length - from);
for (int i = from; i < topDocs.scoreDocs.length; ++i) {
ScoreDoc doc = topDocs.scoreDocs[i];
out.writeVInt(topDocs.scoreDocs.length);
for (ScoreDoc doc : topDocs.scoreDocs) {
writeScoreDoc(out, doc);
}
}

View File

@ -54,6 +54,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.Iterator;
import java.util.Set;
import java.util.concurrent.Callable;
@ -88,6 +89,8 @@ public class IndicesQueryCache extends AbstractComponent implements RemovalListe
public static final String INDICES_CACHE_QUERY_EXPIRE = "indices.cache.query.expire";
public static final String INDICES_CACHE_QUERY_CONCURRENCY_LEVEL = "indices.cache.query.concurrency_level";
private static final Set<SearchType> CACHEABLE_SEARCH_TYPES = EnumSet.of(SearchType.QUERY_THEN_FETCH, SearchType.QUERY_AND_FETCH);
private final ThreadPool threadPool;
private final ClusterService clusterService;
@ -177,10 +180,20 @@ public class IndicesQueryCache extends AbstractComponent implements RemovalListe
if (hasLength(request.templateSource())) {
return false;
}
// for now, only enable it for search type count
if (context.searchType() != SearchType.COUNT) {
// for now, only enable it for requests with no hits
if (context.size() != 0) {
return false;
}
// We cannot cache with DFS because results depend not only on the content of the index but also
// on the overridden statistics. So if you ran two queries on the same index with different stats
// (because an other shard was updated) you would get wrong results because of the scores
// (think about top_hits aggs or scripts using the score)
if (!CACHEABLE_SEARCH_TYPES.contains(context.searchType())) {
return false;
}
IndexMetaData index = clusterService.state().getMetaData().index(request.index());
if (index == null) { // in case we didn't yet have the cluster state, or it just got deleted
return false;

View File

@ -226,17 +226,21 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
public QuerySearchResult executeScan(ShardSearchRequest request) throws ElasticsearchException {
final SearchContext context = createAndPutContext(request);
final int originalSize = context.size();
try {
if (context.aggregations() != null) {
throw new ElasticsearchIllegalArgumentException("aggregations are not supported with search_type=scan");
}
assert context.searchType() == SearchType.SCAN;
context.searchType(SearchType.COUNT); // move to COUNT, and then, when scrolling, move to SCAN
assert context.searchType() == SearchType.COUNT;
if (context.scroll() == null) {
throw new ElasticsearchException("Scroll must be provided when scanning...");
}
assert context.searchType() == SearchType.SCAN;
context.searchType(SearchType.QUERY_THEN_FETCH); // move to QUERY_THEN_FETCH, and then, when scrolling, move to SCAN
context.size(0); // set size to 0 so that we only count matches
assert context.searchType() == SearchType.QUERY_THEN_FETCH;
contextProcessing(context);
queryPhase.execute(context);
contextProcessedSuccessfully(context);
@ -246,6 +250,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
freeContext(context.id());
throw ExceptionsHelper.convertToRuntime(e);
} finally {
context.size(originalSize);
cleanContext(context);
}
}
@ -255,7 +260,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
contextProcessing(context);
try {
processScroll(request, context);
if (context.searchType() == SearchType.COUNT) {
if (context.searchType() == SearchType.QUERY_THEN_FETCH) {
// first scanning, reset the from to 0
context.searchType(SearchType.SCAN);
context.from(0);
@ -300,7 +305,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
loadOrExecuteQueryPhase(request, context, queryPhase);
if (context.searchType() == SearchType.COUNT) {
if (context.queryResult().topDocs().scoreDocs.length == 0) {
freeContext(context.id());
} else {
contextProcessedSuccessfully(context);
@ -357,7 +362,12 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
context.indexShard().searchService().onPreQueryPhase(context);
long time = System.nanoTime();
queryPhase.execute(context);
contextProcessedSuccessfully(context);
if (context.queryResult().topDocs().scoreDocs.length == 0) {
// no hits, we can release the context since there will be no fetch phase
freeContext(context.id());
} else {
contextProcessedSuccessfully(context);
}
context.indexShard().searchService().onQueryPhase(context, System.nanoTime() - time);
return context.queryResult();
} catch (Throwable e) {
@ -377,7 +387,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
context.indexShard().searchService().onPreQueryPhase(context);
long time = System.nanoTime();
try {
queryPhase.execute(context);
loadOrExecuteQueryPhase(request, context, queryPhase);
} catch (Throwable e) {
context.indexShard().searchService().onFailedQueryPhase(context);
throw ExceptionsHelper.convertToRuntime(e);
@ -564,7 +574,12 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
if (context.from() == -1) {
context.from(0);
}
if (context.size() == -1) {
if (context.searchType() == SearchType.COUNT) {
// so that the optimizations we apply to size=0 also apply to search_type=COUNT
// and that we close contexts when done with the query phase
context.searchType(SearchType.QUERY_THEN_FETCH);
context.size(0);
} else if (context.size() == -1) {
context.size(10);
}
@ -992,9 +1007,9 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
SearchType.QUERY_THEN_FETCH, entry.source(), entry.types(), entry.queryCache());
context = createContext(request, warmerContext.searcher());
// if we use sort, we need to do query to sort on it and load relevant field data
// if not, we might as well use COUNT (and cache if needed)
// if not, we might as well set size=0 (and cache if needed)
if (context.sort() == null) {
context.searchType(SearchType.COUNT);
context.size(0);
}
boolean canCache = indicesQueryCache.canCache(request, context);
// early terminate when we can cache, since we can only do proper caching on top level searcher

View File

@ -150,7 +150,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeVInt(from);
out.writeVInt(size);
Lucene.writeTopDocs(out, topDocs, 0);
Lucene.writeTopDocs(out, topDocs);
searchHits.writeTo(out);
}

View File

@ -105,10 +105,10 @@ public class QueryPhase implements SearchPhase {
Query query = searchContext.query();
TopDocs topDocs;
final TopDocs topDocs;
int numDocs = searchContext.from() + searchContext.size();
if (searchContext.searchType() == SearchType.COUNT || numDocs == 0) {
if (searchContext.size() == 0) { // no matter what the value of from is
TotalHitCountCollector collector = new TotalHitCountCollector();
searchContext.searcher().search(query, collector);
topDocs = new TopDocs(collector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0);

View File

@ -180,7 +180,7 @@ public class QuerySearchResult extends QuerySearchResultProvider {
// shardTarget.writeTo(out);
out.writeVInt(from);
out.writeVInt(size);
writeTopDocs(out, topDocs, 0);
writeTopDocs(out, topDocs);
if (aggregations == null) {
out.writeBoolean(false);
} else {

View File

@ -169,13 +169,13 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
req = client.prepareSearch()
.setPreference(suggestions.getPreference())
.setQuery(QueryBuilders.constantScoreQuery(FilterBuilders.bytesFilter(querySource)))
.setSearchType(SearchType.COUNT)
.setSize(0)
.setTerminateAfter(1);
} else {
req = client.prepareSearch()
.setPreference(suggestions.getPreference())
.setQuery(querySource)
.setSearchType(SearchType.COUNT)
.setSize(0)
.setTerminateAfter(1);
}
multiSearchRequestBuilder.add(req);

View File

@ -46,7 +46,7 @@ public class MultiSearchRequestTests extends ElasticsearchTestCase {
assertThat(request.requests().get(2).types().length, equalTo(0));
assertThat(request.requests().get(3).indices(), nullValue());
assertThat(request.requests().get(3).types().length, equalTo(0));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.COUNT));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.DFS_QUERY_THEN_FETCH));
assertThat(request.requests().get(4).indices(), nullValue());
assertThat(request.requests().get(4).types().length, equalTo(0));
}
@ -64,7 +64,7 @@ public class MultiSearchRequestTests extends ElasticsearchTestCase {
assertThat(request.requests().get(2).types().length, equalTo(0));
assertThat(request.requests().get(3).indices(), nullValue());
assertThat(request.requests().get(3).types().length, equalTo(0));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.COUNT));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.DFS_QUERY_THEN_FETCH));
assertThat(request.requests().get(4).indices(), nullValue());
assertThat(request.requests().get(4).types().length, equalTo(0));
}
@ -85,6 +85,6 @@ public class MultiSearchRequestTests extends ElasticsearchTestCase {
assertThat(request.requests().get(2).types()[1], equalTo("type1"));
assertThat(request.requests().get(3).indices(), nullValue());
assertThat(request.requests().get(3).types().length, equalTo(0));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.COUNT));
assertThat(request.requests().get(3).searchType(), equalTo(SearchType.DFS_QUERY_THEN_FETCH));
}
}

View File

@ -4,7 +4,7 @@
{"query" : {"match_all" {}}}
{}
{"query" : {"match_all" {}}}
{"search_type" : "count"}
{"search_type" : "dfs_query_then_fetch"}
{"query" : {"match_all" {}}}
{"query" : {"match_all" {}}}

View File

@ -4,7 +4,7 @@
{"query" : {"match_all" {}}}
{}
{"query" : {"match_all" {}}}
{"search_type" : "count"}
{"search_type" : "dfs_query_then_fetch"}
{"query" : {"match_all" {}}}
{"query" : {"match_all" {}}}

View File

@ -4,5 +4,5 @@
{"query" : {"match_all" {}}}
{"index" : ["test4", "test1"], "type" : [ "type2", "type1" ]}
{"query" : {"match_all" {}}}
{"search_type" : "count"}
{"search_type" : "dfs_query_then_fetch"}
{"query" : {"match_all" {}}}

View File

@ -76,7 +76,7 @@ public class CircuitBreakerBenchmark {
terms("myterms")
.size(AGG_SIZE)
.field("num")
).setSearchType(SearchType.COUNT).get();
).setSize(0).get();
Terms terms = resp.getAggregations().get("myterms");
assertNotNull("term aggs were calculated", terms);
totalTime += resp.getTookInMillis();
@ -103,7 +103,7 @@ public class CircuitBreakerBenchmark {
terms("myterms")
.size(AGG_SIZE)
.field("num")
).setSearchType(SearchType.COUNT).get();
).setSize(0).get();
Terms terms = resp.getAggregations().get("myterms");
assertNotNull("term aggs were calculated", terms);
totalThreadedTime.addAndGet(resp.getTookInMillis());
@ -153,7 +153,7 @@ public class CircuitBreakerBenchmark {
}
bulkBuilder.get();
client.admin().indices().prepareRefresh(INDEX).get();
SearchResponse countResp = client.prepareSearch(INDEX).setQuery(matchAllQuery()).setSearchType(SearchType.COUNT).get();
SearchResponse countResp = client.prepareSearch(INDEX).setQuery(matchAllQuery()).setSize(0).get();
assert countResp.getHits().getTotalHits() == NUM_DOCS : "all docs should be indexed";
final int warmupCount = 100;
@ -166,7 +166,7 @@ public class CircuitBreakerBenchmark {
terms("myterms")
.size(AGG_SIZE)
.field("num")
).setSearchType(SearchType.COUNT).get();
).setSize(0).get();
Terms terms = resp.getAggregations().get("myterms");
assertNotNull("term aggs were calculated", terms);
}

View File

@ -148,7 +148,7 @@ public class CardinalityAggregationSearchBenchmark {
long start = System.nanoTime();
SearchResponse resp = null;
for (int j = 0; j < ITERS; ++j) {
resp = client.prepareSearch("index").setSearchType(SearchType.COUNT).addAggregation(cardinality("cardinality").field(field)).execute().actionGet();
resp = client.prepareSearch("index").setSize(0).addAggregation(cardinality("cardinality").field(field)).execute().actionGet();
}
long end = System.nanoTime();
final long cardinality = ((Cardinality) resp.getAggregations().get("cardinality")).getValue();

View File

@ -211,7 +211,7 @@ public class GlobalOrdinalsBenchmark {
// run just the child query, warm up first
for (int j = 0; j < QUERY_WARMUP; j++) {
SearchResponse searchResponse = client.prepareSearch(INDEX_NAME)
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(AggregationBuilders.terms(name).field(field).executionHint(executionHint))
.get();
@ -229,7 +229,7 @@ public class GlobalOrdinalsBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = client.prepareSearch(INDEX_NAME)
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(AggregationBuilders.terms(name).field(field).executionHint(executionHint))
.get();

View File

@ -177,7 +177,7 @@ public class PercentilesAggregationSearchBenchmark {
}
System.out.println("Expected percentiles: " + percentiles);
System.out.println();
SearchResponse resp = client.prepareSearch(d.indexName()).setSearchType(SearchType.COUNT).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
SearchResponse resp = client.prepareSearch(d.indexName()).setSize(0).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
Percentiles pcts = resp.getAggregations().get("pcts");
Map<Double, Double> asMap = Maps.newLinkedHashMap();
double sumOfErrorSquares = 0;
@ -196,11 +196,11 @@ public class PercentilesAggregationSearchBenchmark {
for (Distribution d : Distribution.values()) {
System.out.println("#### " + d);
for (int j = 0; j < QUERY_WARMUP; ++j) {
client.prepareSearch(d.indexName()).setSearchType(SearchType.COUNT).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
client.prepareSearch(d.indexName()).setSize(0).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
}
long start = System.nanoTime();
for (int j = 0; j < QUERY_COUNT; ++j) {
client.prepareSearch(d.indexName()).setSearchType(SearchType.COUNT).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
client.prepareSearch(d.indexName()).setSize(0).addAggregation(percentiles("pcts").field("v").percentiles(PERCENTILES)).execute().actionGet();
}
System.out.println(new TimeValue((System.nanoTime() - start) / QUERY_COUNT, TimeUnit.NANOSECONDS));
}

View File

@ -126,7 +126,7 @@ public class QueryFilterAggregationSearchBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(termQuery("l_value", anyValue))
.execute().actionGet();
totalQueryTime += searchResponse.getTookInMillis();
@ -136,7 +136,7 @@ public class QueryFilterAggregationSearchBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(termQuery("l_value", anyValue))
.addAggregation(AggregationBuilders.filter("filter").filter(FilterBuilders.termFilter("l_value", anyValue)))
.execute().actionGet();

View File

@ -265,7 +265,7 @@ public class SubAggregationSearchCollectModeBenchmark {
// run just the child query, warm up first
for (int j = 0; j < QUERY_WARMUP; j++) {
SearchResponse searchResponse = client.prepareSearch("test")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(AggregationBuilders.terms(name + "s_value").field("s_value").collectMode(collectionModes[0])
.subAggregation(AggregationBuilders.terms(name + "l_value").field("l_value").collectMode(collectionModes[1])
@ -286,7 +286,7 @@ public class SubAggregationSearchCollectModeBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = client.prepareSearch("test")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(AggregationBuilders.terms(name + "s_value").field("s_value").collectMode(collectionModes[0])
.subAggregation(AggregationBuilders.terms(name + "l_value").field("l_value").collectMode(collectionModes[1])

View File

@ -306,7 +306,7 @@ public class TermsAggregationSearchAndIndexingBenchmark {
while (run) {
try {
SearchResponse searchResponse = Method.AGGREGATION.addTermsAgg(client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery()), "test", field, executionHint)
.execute().actionGet();
if (searchResponse.getHits().totalHits() != COUNT) {

View File

@ -322,7 +322,7 @@ public class TermsAggregationSearchBenchmark {
// run just the child query, warm up first
for (int j = 0; j < QUERY_WARMUP; j++) {
SearchResponse searchResponse = method.addTermsAgg(client.prepareSearch("test")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery()), name, field, executionHint)
.execute().actionGet();
if (j == 0) {
@ -339,7 +339,7 @@ public class TermsAggregationSearchBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = method.addTermsAgg(client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery()), name, field, executionHint)
.execute().actionGet();
if (searchResponse.getHits().totalHits() != COUNT) {
@ -372,7 +372,7 @@ public class TermsAggregationSearchBenchmark {
// run just the child query, warm up first
for (int j = 0; j < QUERY_WARMUP; j++) {
SearchResponse searchResponse = method.addTermsStatsAgg(client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery()), name, keyField, valueField)
.execute().actionGet();
if (j == 0) {
@ -389,7 +389,7 @@ public class TermsAggregationSearchBenchmark {
totalQueryTime = 0;
for (int j = 0; j < QUERY_COUNT; j++) {
SearchResponse searchResponse = method.addTermsStatsAgg(client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery()), name, keyField, valueField)
.execute().actionGet();
if (searchResponse.getHits().totalHits() != COUNT) {

View File

@ -210,7 +210,7 @@ public class TimeDataHistogramAggregationBenchmark {
private static SearchResponse doTermsAggsSearch(String name, String field, float matchPercentage) {
SearchResponse response = client.prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QueryBuilders.constantScoreQuery(FilterBuilders.scriptFilter("random()<matchP").addParam("matchP", matchPercentage).cache(true)))
.addAggregation(AggregationBuilders.histogram(name).field(field).interval(3600 * 1000)).get();

View File

@ -191,7 +191,7 @@ public class GeoDistanceSearchBenchmark {
public static void run(Client client, GeoDistance geoDistance, String optimizeBbox) {
client.prepareSearch() // from NY
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(filteredQuery(matchAllQuery(), geoDistanceFilter("location")
.distance("2km")
.optimizeBbox(optimizeBbox)

View File

@ -71,7 +71,7 @@ public class FieldDataFilterIntegrationTests extends ElasticsearchIntegrationTes
}
refresh();
SearchResponse searchResponse = client().prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(terms("name").field("name"))
.addAggregation(terms("not_filtered").field("not_filtered")).get();

View File

@ -20,7 +20,6 @@
package org.elasticsearch.indices.cache.query;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
@ -48,7 +47,7 @@ public class IndicesQueryCacheTests extends ElasticsearchIntegrationTest {
// This is not a random example: serialization with time zones writes shared strings
// which used to not work well with the query cache because of the handles stream output
// see #9500
final SearchResponse r1 = client().prepareSearch("index").setSearchType(SearchType.COUNT)
final SearchResponse r1 = client().prepareSearch("index").setSize(0)
.addAggregation(dateHistogram("histo").field("f").timeZone("+01:00").minDocCount(0).interval(DateHistogramInterval.MONTH)).get();
assertSearchResponse(r1);
@ -56,7 +55,7 @@ public class IndicesQueryCacheTests extends ElasticsearchIntegrationTest {
assertThat(client().admin().indices().prepareStats("index").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), greaterThan(0l));
for (int i = 0; i < 10; ++i) {
final SearchResponse r2 = client().prepareSearch("index").setSearchType(SearchType.COUNT)
final SearchResponse r2 = client().prepareSearch("index").setSize(0)
.addAggregation(dateHistogram("histo").field("f").timeZone("+01:00").minDocCount(0).interval(DateHistogramInterval.MONTH)).get();
assertSearchResponse(r2);
Histogram h1 = r1.getAggregations().get("histo");

View File

@ -240,7 +240,7 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest {
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getHitCount(), equalTo(0l));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMissCount(), equalTo(0l));
for (int i = 0; i < 10; i++) {
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), greaterThan(0l));
}
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getHitCount(), greaterThan(0l));
@ -265,7 +265,7 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest {
});
for (int i = 0; i < 10; i++) {
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), greaterThan(0l));
}
@ -274,10 +274,10 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest {
// test explicit request parameter
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).setQueryCache(false).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).setQueryCache(false).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), equalTo(0l));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).setQueryCache(true).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).setQueryCache(true).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), greaterThan(0l));
// set the index level setting to false, and see that the reverse works
@ -285,10 +285,10 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest {
client().admin().indices().prepareClearCache().setQueryCache(true).get(); // clean the cache
assertAcked(client().admin().indices().prepareUpdateSettings("idx").setSettings(ImmutableSettings.builder().put(IndicesQueryCache.INDEX_CACHE_QUERY_ENABLED, false)));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), equalTo(0l));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.COUNT).setQueryCache(true).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().prepareSearch("idx").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(0).setQueryCache(true).get().getHits().getTotalHits(), equalTo((long) numDocs));
assertThat(client().admin().indices().prepareStats("idx").setQueryCache(true).get().getTotal().getQueryCache().getMemorySizeInBytes(), greaterThan(0l));
}

View File

@ -275,7 +275,7 @@ public class RecoveryWhileUnderLoadTests extends ElasticsearchIntegrationTest {
SearchResponse[] iterationResults = new SearchResponse[iterations];
boolean error = false;
for (int i = 0; i < iterations; i++) {
SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.COUNT).setQuery(matchAllQuery()).get();
SearchResponse searchResponse = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get();
logSearchResponse(numberOfShards, numberOfDocs, i, searchResponse);
iterationResults[i] = searchResponse;
if (searchResponse.getHits().totalHits() != numberOfDocs) {
@ -298,7 +298,7 @@ public class RecoveryWhileUnderLoadTests extends ElasticsearchIntegrationTest {
public boolean apply(Object o) {
boolean error = false;
for (int i = 0; i < iterations; i++) {
SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.COUNT).setQuery(matchAllQuery()).get();
SearchResponse searchResponse = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get();
if (searchResponse.getHits().totalHits() != numberOfDocs) {
error = true;
}

View File

@ -348,7 +348,7 @@ public class RelocationTests extends ElasticsearchIntegrationTest {
logger.debug("--> verifying all searches return the same number of docs");
long expectedCount = -1;
for (Client client : clients()) {
SearchResponse response = client.prepareSearch("test").setPreference("_local").setSearchType(SearchType.COUNT).get();
SearchResponse response = client.prepareSearch("test").setPreference("_local").setSize(0).get();
assertNoFailures(response);
if (expectedCount < 0) {
expectedCount = response.getHits().totalHits();

View File

@ -0,0 +1,65 @@
/*
* 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;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
/**
* {@link SearchType#COUNT} is deprecated but let's make sure it still works as expected.
*/
public class CountSearchTests extends ElasticsearchIntegrationTest {
public void testDuelCountQueryThenFetch() throws Exception {
createIndex("idx");
ensureYellow();
indexRandom(true,
client().prepareIndex("idx", "type", "1").setSource("foo", "bar", "bar", 3),
client().prepareIndex("idx", "type", "2").setSource("foo", "baz", "bar", 10),
client().prepareIndex("idx", "type", "3").setSource("foo", "foo", "bar", 7));
final SearchResponse resp1 = client().prepareSearch("idx").setSize(0).addAggregation(AggregationBuilders.sum("bar").field("bar")).execute().get();
assertSearchResponse(resp1);
final SearchResponse resp2 = client().prepareSearch("idx").setSearchType(SearchType.COUNT).addAggregation(AggregationBuilders.sum("bar").field("bar")).execute().get();
assertSearchResponse(resp2);
assertEquals(resp1.getHits().getTotalHits(), resp2.getHits().getTotalHits());
Sum sum1 = resp1.getAggregations().get("bar");
Sum sum2 = resp2.getAggregations().get("bar");
assertEquals(sum1.getValue(), sum2.getValue(), 0d);
}
public void testCloseContextEvenWithExplicitSize() throws Exception {
createIndex("idx");
ensureYellow();
indexRandom(true,
client().prepareIndex("idx", "type", "1").setSource("foo", "bar", "bar", 3),
client().prepareIndex("idx", "type", "2").setSource("foo", "baz", "bar", 10),
client().prepareIndex("idx", "type", "3").setSource("foo", "foo", "bar", 7));
client().prepareSearch("idx").setSearchType(SearchType.COUNT).setSize(2).addAggregation(AggregationBuilders.sum("bar").field("bar")).execute().get();
}
}

View File

@ -264,7 +264,7 @@ public class MinDocCountTests extends AbstractTermsTests {
private void testMinDocCountOnTerms(String field, Script script, Terms.Order order, String include, boolean retryOnFailure) throws Exception {
// all terms
final SearchResponse allTermsResponse = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(script.apply(terms("terms"), field)
.collectMode(randomFrom(SubAggCollectionMode.values()))
@ -281,7 +281,7 @@ public class MinDocCountTests extends AbstractTermsTests {
for (long minDocCount = 0; minDocCount < 20; ++minDocCount) {
final int size = randomIntBetween(1, cardinality + 2);
final SearchRequest request = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(script.apply(terms("terms"), field)
.collectMode(randomFrom(SubAggCollectionMode.values()))
@ -349,7 +349,7 @@ public class MinDocCountTests extends AbstractTermsTests {
private void testMinDocCountOnHistogram(Histogram.Order order) throws Exception {
final int interval = randomIntBetween(1, 3);
final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(histogram("histo").field("d").interval(interval).order(order).minDocCount(0))
.execute().actionGet();
@ -358,7 +358,7 @@ public class MinDocCountTests extends AbstractTermsTests {
for (long minDocCount = 0; minDocCount < 50; ++minDocCount) {
final SearchResponse response = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(histogram("histo").field("d").interval(interval).order(order).minDocCount(minDocCount))
.execute().actionGet();
@ -370,7 +370,7 @@ public class MinDocCountTests extends AbstractTermsTests {
private void testMinDocCountOnDateHistogram(Histogram.Order order) throws Exception {
final int interval = randomIntBetween(1, 3);
final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(dateHistogram("histo").field("date").interval(DateHistogramInterval.DAY).order(order).minDocCount(0))
.execute().actionGet();
@ -379,7 +379,7 @@ public class MinDocCountTests extends AbstractTermsTests {
for (long minDocCount = 0; minDocCount < 50; ++minDocCount) {
final SearchResponse response = client().prepareSearch("idx").setTypes("type")
.setSearchType(SearchType.COUNT)
.setSize(0)
.setQuery(QUERY)
.addAggregation(dateHistogram("histo").field("date").interval(DateHistogramInterval.DAY).order(order).minDocCount(minDocCount))
.execute().actionGet();

View File

@ -664,7 +664,7 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
logger.info("Now testing GeoDistance={}, distance={}, origin=({}, {})", geoDistance, distance, originLat, originLon);
long matches = -1;
for (String optimizeBbox : Arrays.asList("none", "memory", "indexed")) {
SearchResponse resp = client().prepareSearch("index").setSearchType(SearchType.COUNT).setQuery(QueryBuilders.constantScoreQuery(
SearchResponse resp = client().prepareSearch("index").setSize(0).setQuery(QueryBuilders.constantScoreQuery(
FilterBuilders.geoDistanceFilter("location").point(originLat, originLon).distance(distance).geoDistance(geoDistance).optimizeBbox(optimizeBbox))).execute().actionGet();
assertSearchResponse(resp);
logger.info("{} -> {} hits", optimizeBbox, resp.getHits().totalHits());

View File

@ -50,7 +50,7 @@ public class SearchPreferenceTests extends ElasticsearchIntegrationTest {
client().admin().cluster().prepareHealth().setWaitForStatus(ClusterHealthStatus.RED).execute().actionGet();
String[] preferences = new String[] {"_primary", "_local", "_primary_first", "_prefer_node:somenode", "_prefer_node:server2"};
for (String pref : preferences) {
SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.COUNT).setPreference(pref).execute().actionGet();
SearchResponse searchResponse = client().prepareSearch().setSize(0).setPreference(pref).execute().actionGet();
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
assertThat(pref, searchResponse.getFailedShards(), greaterThanOrEqualTo(0));
searchResponse = client().prepareSearch().setPreference(pref).execute().actionGet();
@ -59,7 +59,7 @@ public class SearchPreferenceTests extends ElasticsearchIntegrationTest {
}
//_only_local is a stricter preference, we need to send the request to a data node
SearchResponse searchResponse = dataNodeClient().prepareSearch().setSearchType(SearchType.COUNT).setPreference("_only_local").execute().actionGet();
SearchResponse searchResponse = dataNodeClient().prepareSearch().setSize(0).setPreference("_only_local").execute().actionGet();
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
assertThat("_only_local", searchResponse.getFailedShards(), greaterThanOrEqualTo(0));
searchResponse = dataNodeClient().prepareSearch().setPreference("_only_local").execute().actionGet();

View File

@ -66,6 +66,7 @@ public class SearchScanTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
assertThat(searchResponse.getHits().totalHits(), equalTo((long)builders.length/2));
assertThat(searchResponse.getHits().getHits().length, equalTo(0));
// start scrolling, until we get not results
while (true) {

View File

@ -427,7 +427,7 @@ public class SearchScrollTests extends ElasticsearchIntegrationTest {
.setQuery(QueryBuilders.matchAllQuery())
.setSize(Integer.MAX_VALUE);
if (searchType == SearchType.SCAN || searchType != SearchType.COUNT && randomBoolean()) {
if (searchType == SearchType.SCAN || randomBoolean()) {
builder.setScroll("1m");
}

View File

@ -272,13 +272,13 @@ public class SuggestSearchTests extends ElasticsearchIntegrationTest {
phraseSuggestion.field("nosuchField");
{
SearchRequestBuilder suggestBuilder = client().prepareSearch().setSearchType(SearchType.COUNT);
SearchRequestBuilder suggestBuilder = client().prepareSearch().setSize(0);
suggestBuilder.setSuggestText("tetsting sugestion");
suggestBuilder.addSuggestion(phraseSuggestion);
assertThrows(suggestBuilder, SearchPhaseExecutionException.class);
}
{
SearchRequestBuilder suggestBuilder = client().prepareSearch().setSearchType(SearchType.COUNT);
SearchRequestBuilder suggestBuilder = client().prepareSearch().setSize(0);
suggestBuilder.setSuggestText("tetsting sugestion");
suggestBuilder.addSuggestion(phraseSuggestion);
assertThrows(suggestBuilder, SearchPhaseExecutionException.class);
@ -815,13 +815,13 @@ public class SuggestSearchTests extends ElasticsearchIntegrationTest {
refresh();
// When searching on a shard with a non existing mapping, we should fail
SearchRequestBuilder request = client().prepareSearch().setSearchType(SearchType.COUNT)
SearchRequestBuilder request = client().prepareSearch().setSize(0)
.setSuggestText("tetsting sugestion")
.addSuggestion(phraseSuggestion("did_you_mean").field("fielddoesnotexist").maxErrors(5.0f));
assertThrows(request, SearchPhaseExecutionException.class);
// When searching on a shard which does not hold yet any document of an existing type, we should not fail
SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.COUNT)
SearchResponse searchResponse = client().prepareSearch().setSize(0)
.setSuggestText("tetsting sugestion")
.addSuggestion(phraseSuggestion("did_you_mean").field("name").maxErrors(5.0f))
.get();
@ -864,7 +864,7 @@ public class SuggestSearchTests extends ElasticsearchIntegrationTest {
refresh();
SearchResponse searchResponse = client().prepareSearch()
.setSearchType(SearchType.COUNT)
.setSize(0)
.setSuggestText("tetsting sugestion")
.addSuggestion(phraseSuggestion("did_you_mean").field("name").maxErrors(5.0f))
.get();
@ -1268,7 +1268,7 @@ public class SuggestSearchTests extends ElasticsearchIntegrationTest {
protected Suggest searchSuggest(String suggestText, int expectShardsFailed, SuggestionBuilder<?>... suggestions) {
if (randomBoolean()) {
SearchRequestBuilder builder = client().prepareSearch().setSearchType(SearchType.COUNT);
SearchRequestBuilder builder = client().prepareSearch().setSize(0);
if (suggestText != null) {
builder.setSuggestText(suggestText);
}