From 0c26e7cd834dfdbf2148f09f234d6c80f1f3941e Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 3 Sep 2015 15:00:52 +0200 Subject: [PATCH] Remove the scan and count search types. These search types have been deprecated in 2.1 and 2.0 respectively, and will be removed in 3.0. --- .../elasticsearch/action/ActionModule.java | 6 +- .../action/search/SearchType.java | 29 +-- .../action/search/TransportSearchAction.java | 12 +- .../search/TransportSearchScrollAction.java | 7 +- .../action/search/type/ParsedScrollId.java | 2 - .../type/TransportSearchCountAction.java | 87 ------- .../search/type/TransportSearchHelper.java | 2 - .../type/TransportSearchScanAction.java | 86 ------- .../type/TransportSearchScrollScanAction.java | 238 ------------------ .../percolator/PercolateContext.java | 6 - .../elasticsearch/search/SearchService.java | 85 +------ .../action/SearchServiceTransportAction.java | 38 --- .../search/internal/DefaultSearchContext.java | 12 - .../internal/FilteredSearchContext.java | 6 - .../search/internal/SearchContext.java | 3 - .../search/query/QueryPhase.java | 11 - .../search/scan/ScanContext.java | 116 --------- .../action/IndicesRequestIT.java | 22 -- .../BasicBackwardsCompatibilityIT.java | 13 +- .../elasticsearch/search/CountSearchIT.java | 65 ----- .../AggregationsIntegrationIT.java | 11 - .../search/child/ChildQuerySearchIT.java | 6 +- .../search/scan/ScanContextTests.java | 96 ------- .../search/scan/SearchScanIT.java | 85 ------- .../search/scan/SearchScanScrollingIT.java | 106 -------- .../search/scroll/SearchScrollIT.java | 2 +- .../search/stats/SearchStatsIT.java | 22 +- .../elasticsearch/test/TestSearchContext.java | 6 - docs/reference/migration/index.asciidoc | 2 + docs/reference/migration/migrate_3_0.asciidoc | 65 +++++ .../reference/search/request/rescore.asciidoc | 4 - .../search/request/search-type.asciidoc | 22 -- docs/reference/search/uri-request.asciidoc | 6 +- .../TransportDeleteByQueryActionTests.java | 34 ++- .../resources/rest-api-spec/api/msearch.json | 2 +- .../resources/rest-api-spec/api/search.json | 2 +- .../rest-api-spec/api/search_template.json | 2 +- .../rest-api-spec/test/scroll/10_basic.yaml | 65 ----- .../rest-api-spec/test/scroll/11_clear.yaml | 2 - .../test/search/50_search_count.yaml | 33 --- 40 files changed, 115 insertions(+), 1304 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/action/search/type/TransportSearchCountAction.java delete mode 100644 core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScanAction.java delete mode 100644 core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java delete mode 100644 core/src/main/java/org/elasticsearch/search/scan/ScanContext.java delete mode 100644 core/src/test/java/org/elasticsearch/search/CountSearchIT.java delete mode 100644 core/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java delete mode 100644 core/src/test/java/org/elasticsearch/search/scan/SearchScanIT.java delete mode 100644 core/src/test/java/org/elasticsearch/search/scan/SearchScanScrollingIT.java create mode 100644 docs/reference/migration/migrate_3_0.asciidoc delete mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search/50_search_count.yaml diff --git a/core/src/main/java/org/elasticsearch/action/ActionModule.java b/core/src/main/java/org/elasticsearch/action/ActionModule.java index 7be0b032259..bcb30ded360 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/core/src/main/java/org/elasticsearch/action/ActionModule.java @@ -171,10 +171,8 @@ import org.elasticsearch.action.search.type.TransportSearchDfsQueryAndFetchActio import org.elasticsearch.action.search.type.TransportSearchDfsQueryThenFetchAction; import org.elasticsearch.action.search.type.TransportSearchQueryAndFetchAction; import org.elasticsearch.action.search.type.TransportSearchQueryThenFetchAction; -import org.elasticsearch.action.search.type.TransportSearchScanAction; import org.elasticsearch.action.search.type.TransportSearchScrollQueryAndFetchAction; import org.elasticsearch.action.search.type.TransportSearchScrollQueryThenFetchAction; -import org.elasticsearch.action.search.type.TransportSearchScrollScanAction; import org.elasticsearch.action.suggest.SuggestAction; import org.elasticsearch.action.suggest.TransportSuggestAction; import org.elasticsearch.action.support.ActionFilter; @@ -327,11 +325,9 @@ public class ActionModule extends AbstractModule { TransportSearchDfsQueryThenFetchAction.class, TransportSearchQueryThenFetchAction.class, TransportSearchDfsQueryAndFetchAction.class, - TransportSearchQueryAndFetchAction.class, - TransportSearchScanAction.class + TransportSearchQueryAndFetchAction.class ); registerAction(SearchScrollAction.INSTANCE, TransportSearchScrollAction.class, - TransportSearchScrollScanAction.class, TransportSearchScrollQueryThenFetchAction.class, TransportSearchScrollQueryAndFetchAction.class ); diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchType.java b/core/src/main/java/org/elasticsearch/action/search/SearchType.java index 6d91e40f243..31535736957 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchType.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchType.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.search; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseFieldMatcher; /** @@ -50,29 +49,13 @@ public enum SearchType { * and return the results. Each shard returns size results. Since each shard already returns size hits, this * type actually returns size times number of shards results back to the caller. */ - QUERY_AND_FETCH((byte) 3), - /** - * Performs scanning of the results which executes the search without any sorting. - * It will automatically start scrolling the result set. - * @deprecated will be removed in 3.0, you should do a regular scroll instead, ordered by `_doc` - */ - @Deprecated - 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); + QUERY_AND_FETCH((byte) 3); /** * The default search type ({@link #QUERY_THEN_FETCH}. */ public static final SearchType DEFAULT = QUERY_THEN_FETCH; - private static final ParseField COUNT_VALUE = new ParseField("count").withAllDeprecated("query_then_fetch"); - private static final ParseField SCAN_VALUE = new ParseField("scan").withAllDeprecated("query_then_fetch sorting on `_doc`"); - private byte id; SearchType(byte id) { @@ -98,10 +81,6 @@ public enum SearchType { return DFS_QUERY_AND_FETCH; } else if (id == 3) { return QUERY_AND_FETCH; - } else if (id == 4) { - return SCAN; - } else if (id == 5) { - return COUNT; } else { throw new IllegalArgumentException("No search type for [" + id + "]"); } @@ -110,7 +89,7 @@ public enum SearchType { /** * The a string representation search type to execute, defaults to {@link SearchType#DEFAULT}. Can be * one of "dfs_query_then_fetch"/"dfsQueryThenFetch", "dfs_query_and_fetch"/"dfsQueryAndFetch", - * "query_then_fetch"/"queryThenFetch", "query_and_fetch"/"queryAndFetch", and "scan". + * "query_then_fetch"/"queryThenFetch" and "query_and_fetch"/"queryAndFetch". */ public static SearchType fromString(String searchType, ParseFieldMatcher parseFieldMatcher) { if (searchType == null) { @@ -124,10 +103,6 @@ public enum SearchType { return SearchType.QUERY_THEN_FETCH; } else if ("query_and_fetch".equals(searchType)) { return SearchType.QUERY_AND_FETCH; - } else if (parseFieldMatcher.match(searchType, SCAN_VALUE)) { - return SearchType.SCAN; - } else if (parseFieldMatcher.match(searchType, COUNT_VALUE)) { - return SearchType.COUNT; } else { throw new IllegalArgumentException("No search type for [" + searchType + "]"); } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index fda34657f11..bb04ae2ed14 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -48,8 +48,6 @@ public class TransportSearchAction extends HandledTransportAction listener) { // optimize search type for cases where there is only one shard group to search on - if (optimizeSingleShard && searchRequest.searchType() != SCAN && searchRequest.searchType() != COUNT) { + if (optimizeSingleShard) { try { ClusterState clusterState = clusterService.state(); String[] concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, searchRequest); @@ -101,10 +95,6 @@ public class TransportSearchAction extends HandledTransportAction listener) { - new AsyncAction(searchRequest, listener).start(); - } - - private class AsyncAction extends BaseAsyncAction { - - private AsyncAction(SearchRequest request, ActionListener listener) { - super(request, listener); - } - - @Override - protected String firstPhaseName() { - return "query"; - } - - @Override - protected void sendExecuteFirstPhase(DiscoveryNode node, ShardSearchTransportRequest request, ActionListener listener) { - searchService.sendExecuteQuery(node, request, listener); - } - - @Override - protected void moveToSecondPhase() throws Exception { - // no need to sort, since we know we have no hits back - final InternalSearchResponse internalResponse = searchPhaseController.merge(SearchPhaseController.EMPTY_DOCS, firstResults, - (AtomicArray) AtomicArray.empty(), request); - String scrollId = null; - if (request.scroll() != null) { - scrollId = buildScrollId(request.searchType(), firstResults, null); - } - listener.onResponse(new SearchResponse(internalResponse, scrollId, expectedSuccessfulOps, successfulOps.get(), buildTookInMillis(), buildShardFailures())); - } - } -} diff --git a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java index bb9e85810a2..624edbc173d 100644 --- a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java +++ b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java @@ -56,8 +56,6 @@ public abstract class TransportSearchHelper { return buildScrollId(ParsedScrollId.QUERY_THEN_FETCH_TYPE, searchPhaseResults, attributes); } else if (searchType == SearchType.QUERY_AND_FETCH || searchType == SearchType.DFS_QUERY_AND_FETCH) { return buildScrollId(ParsedScrollId.QUERY_AND_FETCH_TYPE, searchPhaseResults, attributes); - } else if (searchType == SearchType.SCAN) { - return buildScrollId(ParsedScrollId.SCAN, searchPhaseResults, attributes); } else { throw new IllegalStateException("search_type [" + searchType + "] not supported"); } diff --git a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScanAction.java b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScanAction.java deleted file mode 100644 index cf2b4ee8df0..00000000000 --- a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScanAction.java +++ /dev/null @@ -1,86 +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.action.search.type; - -import com.google.common.collect.ImmutableMap; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.AtomicArray; -import org.elasticsearch.search.action.SearchServiceTransportAction; -import org.elasticsearch.search.controller.SearchPhaseController; -import org.elasticsearch.search.fetch.FetchSearchResultProvider; -import org.elasticsearch.search.internal.InternalSearchResponse; -import org.elasticsearch.search.internal.ShardSearchTransportRequest; -import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.threadpool.ThreadPool; - -import static org.elasticsearch.action.search.type.TransportSearchHelper.buildScrollId; - -@Deprecated // remove in 3.0 -public class TransportSearchScanAction extends TransportSearchTypeAction { - - @Inject - public TransportSearchScanAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, - SearchServiceTransportAction searchService, SearchPhaseController searchPhaseController, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { - super(settings, threadPool, clusterService, searchService, searchPhaseController, actionFilters, indexNameExpressionResolver); - } - - @Override - protected void doExecute(SearchRequest searchRequest, ActionListener listener) { - new AsyncAction(searchRequest, listener).start(); - } - - private class AsyncAction extends BaseAsyncAction { - - private AsyncAction(SearchRequest request, ActionListener listener) { - super(request, listener); - } - - @Override - protected String firstPhaseName() { - return "init_scan"; - } - - @Override - protected void sendExecuteFirstPhase(DiscoveryNode node, ShardSearchTransportRequest request, ActionListener listener) { - searchService.sendExecuteScan(node, request, listener); - } - - @Override - protected void moveToSecondPhase() throws Exception { - final InternalSearchResponse internalResponse = searchPhaseController.merge(SearchPhaseController.EMPTY_DOCS, firstResults, - (AtomicArray) AtomicArray.empty(), request); - String scrollId = null; - if (request.scroll() != null) { - scrollId = buildScrollId(request.searchType(), firstResults, ImmutableMap.of("total_hits", Long.toString(internalResponse.hits().totalHits()))); - } - listener.onResponse(new SearchResponse(internalResponse, scrollId, expectedSuccessfulOps, successfulOps.get(), buildTookInMillis(), buildShardFailures())); - } - } -} diff --git a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java deleted file mode 100644 index 2bc516b2bbe..00000000000 --- a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java +++ /dev/null @@ -1,238 +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.action.search.type; - -import org.apache.lucene.search.ScoreDoc; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.ReduceSearchPhaseException; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollRequest; -import org.elasticsearch.action.search.ShardSearchFailure; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.AtomicArray; -import org.elasticsearch.search.action.SearchServiceTransportAction; -import org.elasticsearch.search.controller.SearchPhaseController; -import org.elasticsearch.search.fetch.QueryFetchSearchResult; -import org.elasticsearch.search.fetch.ScrollQueryFetchSearchResult; -import org.elasticsearch.search.internal.InternalSearchHits; -import org.elasticsearch.search.internal.InternalSearchResponse; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest; - -/** - * - */ -public class TransportSearchScrollScanAction extends AbstractComponent { - - private final ClusterService clusterService; - private final SearchServiceTransportAction searchService; - private final SearchPhaseController searchPhaseController; - - @Inject - public TransportSearchScrollScanAction(Settings settings, ClusterService clusterService, - SearchServiceTransportAction searchService, SearchPhaseController searchPhaseController) { - super(settings); - this.clusterService = clusterService; - this.searchService = searchService; - this.searchPhaseController = searchPhaseController; - } - - public void execute(SearchScrollRequest request, ParsedScrollId scrollId, ActionListener listener) { - new AsyncAction(request, scrollId, listener).start(); - } - - private class AsyncAction extends AbstractAsyncAction { - - private final SearchScrollRequest request; - - private final ActionListener listener; - - private final ParsedScrollId scrollId; - - private final DiscoveryNodes nodes; - - private volatile AtomicArray shardFailures; - private final AtomicArray queryFetchResults; - - private final AtomicInteger successfulOps; - private final AtomicInteger counter; - - private AsyncAction(SearchScrollRequest request, ParsedScrollId scrollId, ActionListener listener) { - this.request = request; - this.listener = listener; - this.scrollId = scrollId; - this.nodes = clusterService.state().nodes(); - this.successfulOps = new AtomicInteger(scrollId.getContext().length); - this.counter = new AtomicInteger(scrollId.getContext().length); - - this.queryFetchResults = new AtomicArray<>(scrollId.getContext().length); - } - - protected final ShardSearchFailure[] buildShardFailures() { - if (shardFailures == null) { - return ShardSearchFailure.EMPTY_ARRAY; - } - List> entries = shardFailures.asList(); - ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; - for (int i = 0; i < failures.length; i++) { - failures[i] = entries.get(i).value; - } - return failures; - } - - // we do our best to return the shard failures, but its ok if its not fully concurrently safe - // we simply try and return as much as possible - protected final void addShardFailure(final int shardIndex, ShardSearchFailure failure) { - if (shardFailures == null) { - shardFailures = new AtomicArray<>(scrollId.getContext().length); - } - shardFailures.set(shardIndex, failure); - } - - public void start() { - if (scrollId.getContext().length == 0) { - final InternalSearchResponse internalResponse = new InternalSearchResponse(new InternalSearchHits(InternalSearchHits.EMPTY, Long.parseLong(this.scrollId.getAttributes().get("total_hits")), 0.0f), null, null, false, null); - listener.onResponse(new SearchResponse(internalResponse, request.scrollId(), 0, 0, 0l, buildShardFailures())); - return; - } - - ScrollIdForNode[] context = scrollId.getContext(); - for (int i = 0; i < context.length; i++) { - ScrollIdForNode target = context[i]; - DiscoveryNode node = nodes.get(target.getNode()); - if (node != null) { - executePhase(i, node, target.getScrollId()); - } else { - if (logger.isDebugEnabled()) { - logger.debug("Node [" + target.getScrollId() + "] not available for scroll request [" + scrollId.getSource() + "]"); - } - successfulOps.decrementAndGet(); - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } - } - - for (ScrollIdForNode target : scrollId.getContext()) { - DiscoveryNode node = nodes.get(target.getNode()); - if (node == null) { - if (logger.isDebugEnabled()) { - logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]"); - } - successfulOps.decrementAndGet(); - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } else { - } - } - } - - void executePhase(final int shardIndex, DiscoveryNode node, final long searchId) { - searchService.sendExecuteScan(node, internalScrollSearchRequest(searchId, request), new ActionListener() { - @Override - public void onResponse(ScrollQueryFetchSearchResult result) { - QueryFetchSearchResult shardResult = result.result(); - Objects.requireNonNull(shardResult, "QueryFetchSearchResult can't be null"); - queryFetchResults.setOnce(shardIndex, shardResult); - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } - - @Override - public void onFailure(Throwable t) { - onPhaseFailure(t, searchId, shardIndex); - } - }); - } - - void onPhaseFailure(Throwable t, long searchId, int shardIndex) { - if (logger.isDebugEnabled()) { - logger.debug("[{}] Failed to execute query phase", t, searchId); - } - addShardFailure(shardIndex, new ShardSearchFailure(t)); - successfulOps.decrementAndGet(); - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } - - private void finishHim() { - try { - innerFinishHim(); - } catch (Throwable e) { - ReduceSearchPhaseException failure = new ReduceSearchPhaseException("fetch", "", e, buildShardFailures()); - if (logger.isDebugEnabled()) { - logger.debug("failed to reduce search", failure); - } - listener.onFailure(failure); - } - } - - private void innerFinishHim() throws IOException { - int numberOfHits = 0; - List> entries = queryFetchResults.asList(); - for (AtomicArray.Entry entry : entries) { - numberOfHits += entry.value.queryResult().topDocs().scoreDocs.length; - } - List docs = new ArrayList<>(numberOfHits); - for (AtomicArray.Entry entry : entries) { - ScoreDoc[] scoreDocs = entry.value.queryResult().topDocs().scoreDocs; - for (ScoreDoc scoreDoc : scoreDocs) { - scoreDoc.shardIndex = entry.index; - docs.add(scoreDoc); - } - } - final InternalSearchResponse internalResponse = searchPhaseController.merge(docs.toArray(new ScoreDoc[0]), queryFetchResults, - queryFetchResults, request); - ((InternalSearchHits) internalResponse.hits()).totalHits = Long.parseLong(this.scrollId.getAttributes().get("total_hits")); - - - for (AtomicArray.Entry entry : entries) { - if (entry.value.queryResult().topDocs().scoreDocs.length < entry.value.queryResult().size()) { - // we found more than we want for this round, remove this from our scrolling, so we don't go back to - // this shard, since all hits have been processed. - // The SearchContext already gets freed on the node holding the shard, via a similar check. - queryFetchResults.set(entry.index, null); - } - } - - String scrollId = null; - if (request.scroll() != null) { - // we rebuild the scroll id since we remove shards that we finished scrolling on - scrollId = TransportSearchHelper.buildScrollId(this.scrollId.getType(), queryFetchResults, this.scrollId.getAttributes()); // continue moving the total_hits - } - listener.onResponse(new SearchResponse(internalResponse, scrollId, this.scrollId.getContext().length, successfulOps.get(), - buildTookInMillis(), buildShardFailures())); - } - } -} diff --git a/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java b/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java index c54cce7d361..344bc31c2b4 100644 --- a/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java +++ b/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java @@ -75,7 +75,6 @@ import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.util.Collections; @@ -641,11 +640,6 @@ public class PercolateContext extends SearchContext { throw new UnsupportedOperationException(); } - @Override - public ScanContext scanContext() { - throw new UnsupportedOperationException(); - } - @Override public MappedFieldType smartNameFieldType(String name) { return mapperService().smartNameFieldType(name, types); diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 4288e1d098a..ea1eb710115 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -28,7 +28,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.search.TopDocs; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.search.SearchType; @@ -277,83 +276,6 @@ public class SearchService extends AbstractLifecycleComponent { } } - @Deprecated // remove in 3.0 - public QuerySearchResult executeScan(ShardSearchRequest request) { - final SearchContext context = createAndPutContext(request); - final int originalSize = context.size(); - try { - if (context.aggregations() != null) { - throw new IllegalArgumentException("aggregations are not supported with search_type=scan"); - } - - if (context.scrollContext() == null || context.scrollContext().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); - return context.queryResult(); - } catch (Throwable e) { - logger.trace("Scan phase failed", e); - processFailure(context, e); - throw ExceptionsHelper.convertToRuntime(e); - } finally { - context.size(originalSize); - cleanContext(context); - } - } - - public ScrollQueryFetchSearchResult executeScan(InternalScrollSearchRequest request) { - final SearchContext context = findContext(request.id()); - ShardSearchStats shardSearchStats = context.indexShard().searchService(); - contextProcessing(context); - try { - processScroll(request, context); - shardSearchStats.onPreQueryPhase(context); - long time = System.nanoTime(); - try { - if (context.searchType() == SearchType.QUERY_THEN_FETCH) { - // first scanning, reset the from to 0 - context.searchType(SearchType.SCAN); - context.from(0); - } - queryPhase.execute(context); - } catch (Throwable e) { - shardSearchStats.onFailedQueryPhase(context); - throw ExceptionsHelper.convertToRuntime(e); - } - long queryFinishTime = System.nanoTime(); - shardSearchStats.onQueryPhase(context, queryFinishTime - time); - shardSearchStats.onPreFetchPhase(context); - try { - shortcutDocIdsToLoadForScanning(context); - fetchPhase.execute(context); - if (context.scrollContext() == null || context.fetchResult().hits().hits().length < context.size()) { - freeContext(request.id()); - } else { - contextProcessedSuccessfully(context); - } - } catch (Throwable e) { - shardSearchStats.onFailedFetchPhase(context); - throw ExceptionsHelper.convertToRuntime(e); - } - shardSearchStats.onFetchPhase(context, System.nanoTime() - queryFinishTime); - return new ScrollQueryFetchSearchResult(new QueryFetchSearchResult(context.queryResult(), context.fetchResult()), context.shardTarget()); - } catch (Throwable e) { - logger.trace("Scan phase failed", e); - processFailure(context, e); - throw ExceptionsHelper.convertToRuntime(e); - } finally { - cleanContext(context); - } - } - /** * Try to load the query results from the cache or execute the query phase directly if the cache cannot be used. */ @@ -658,12 +580,7 @@ public class SearchService extends AbstractLifecycleComponent { if (context.from() == -1) { context.from(0); } - 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) { + if (context.size() == -1) { context.size(10); } diff --git a/core/src/main/java/org/elasticsearch/search/action/SearchServiceTransportAction.java b/core/src/main/java/org/elasticsearch/search/action/SearchServiceTransportAction.java index 4205fd95299..4ab0543bcaf 100644 --- a/core/src/main/java/org/elasticsearch/search/action/SearchServiceTransportAction.java +++ b/core/src/main/java/org/elasticsearch/search/action/SearchServiceTransportAction.java @@ -64,8 +64,6 @@ public class SearchServiceTransportAction extends AbstractComponent { public static final String QUERY_FETCH_SCROLL_ACTION_NAME = "indices:data/read/search[phase/query+fetch/scroll]"; public static final String FETCH_ID_SCROLL_ACTION_NAME = "indices:data/read/search[phase/fetch/id/scroll]"; public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; - public static final String SCAN_ACTION_NAME = "indices:data/read/search[phase/scan]"; - public static final String SCAN_SCROLL_ACTION_NAME = "indices:data/read/search[phase/scan/scroll]"; private final TransportService transportService; private final SearchService searchService; @@ -88,8 +86,6 @@ public class SearchServiceTransportAction extends AbstractComponent { transportService.registerRequestHandler(QUERY_FETCH_SCROLL_ACTION_NAME, InternalScrollSearchRequest.class, ThreadPool.Names.SEARCH, new SearchQueryFetchScrollTransportHandler()); transportService.registerRequestHandler(FETCH_ID_SCROLL_ACTION_NAME, ShardFetchRequest.class, ThreadPool.Names.SEARCH, new FetchByIdTransportHandler<>()); transportService.registerRequestHandler(FETCH_ID_ACTION_NAME, ShardFetchSearchRequest.class, ThreadPool.Names.SEARCH, new FetchByIdTransportHandler()); - transportService.registerRequestHandler(SCAN_ACTION_NAME, ShardSearchTransportRequest.class, ThreadPool.Names.SEARCH, new SearchScanTransportHandler()); - transportService.registerRequestHandler(SCAN_SCROLL_ACTION_NAME, InternalScrollSearchRequest.class, ThreadPool.Names.SEARCH, new SearchScanScrollTransportHandler()); } public void sendFreeContext(DiscoveryNode node, final long contextId, SearchRequest request) { @@ -209,24 +205,6 @@ public class SearchServiceTransportAction extends AbstractComponent { }); } - public void sendExecuteScan(DiscoveryNode node, final ShardSearchTransportRequest request, final ActionListener listener) { - transportService.sendRequest(node, SCAN_ACTION_NAME, request, new ActionListenerResponseHandler(listener) { - @Override - public QuerySearchResult newInstance() { - return new QuerySearchResult(); - } - }); - } - - public void sendExecuteScan(DiscoveryNode node, final InternalScrollSearchRequest request, final ActionListener listener) { - transportService.sendRequest(node, SCAN_SCROLL_ACTION_NAME, request, new ActionListenerResponseHandler(listener) { - @Override - public ScrollQueryFetchSearchResult newInstance() { - return new ScrollQueryFetchSearchResult(); - } - }); - } - static class ScrollFreeContextRequest extends TransportRequest { private long id; @@ -418,20 +396,4 @@ public class SearchServiceTransportAction extends AbstractComponent { } } - @Deprecated // remove in 3.0 - class SearchScanTransportHandler implements TransportRequestHandler { - @Override - public void messageReceived(ShardSearchTransportRequest request, TransportChannel channel) throws Exception { - QuerySearchResult result = searchService.executeScan(request); - channel.sendResponse(result); - } - } - - class SearchScanScrollTransportHandler implements TransportRequestHandler { - @Override - public void messageReceived(InternalScrollSearchRequest request, TransportChannel channel) throws Exception { - ScrollQueryFetchSearchResult result = searchService.executeScan(request); - channel.sendResponse(result); - } - } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index 435bd219509..d3ea19e2098 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -64,7 +64,6 @@ import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QueryPhaseExecutionException; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.io.IOException; @@ -94,8 +93,6 @@ public class DefaultSearchContext extends SearchContext { private final DfsSearchResult dfsResult; private final QuerySearchResult queryResult; private final FetchSearchResult fetchResult; - // lazy initialized only if needed - private ScanContext scanContext; private float queryBoost = 1.0f; // timeout in millis private long timeoutInMillis; @@ -162,7 +159,6 @@ public class DefaultSearchContext extends SearchContext { @Override public void doClose() { - scanContext = null; // clear and scope phase we have Releasables.close(searcher, engineSearcher); } @@ -681,14 +677,6 @@ public class DefaultSearchContext extends SearchContext { return fetchResult; } - @Override - public ScanContext scanContext() { - if (scanContext == null) { - scanContext = new ScanContext(); - } - return scanContext; - } - @Override public MappedFieldType smartNameFieldType(String name) { return mapperService().smartNameFieldType(name, request.types()); diff --git a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index a4b9c4d852b..9eb43611538 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -51,7 +51,6 @@ import org.elasticsearch.search.highlight.SearchContextHighlight; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.util.List; @@ -497,11 +496,6 @@ public abstract class FilteredSearchContext extends SearchContext { return in.fetchResult(); } - @Override - public ScanContext scanContext() { - return in.scanContext(); - } - @Override public MappedFieldType smartNameFieldType(String name) { return in.smartNameFieldType(name); diff --git a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 6b6c4225861..234a841e017 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -60,7 +60,6 @@ import org.elasticsearch.search.highlight.SearchContextHighlight; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.util.ArrayList; @@ -335,8 +334,6 @@ public abstract class SearchContext extends DelegatingHasContextAndHeaders imple } } - public abstract ScanContext scanContext(); - public abstract MappedFieldType smartNameFieldType(String name); /** diff --git a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java index fd82ce0ef57..01b9913614d 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -38,7 +38,6 @@ import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.rescore.RescorePhase; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext.ScanCollector; import org.elasticsearch.search.sort.SortParseElement; import org.elasticsearch.search.sort.TrackScoresParseElement; import org.elasticsearch.search.suggest.SuggestPhase; @@ -157,16 +156,6 @@ public class QueryPhase implements SearchPhase { return new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0); } }; - } else if (searchType == SearchType.SCAN) { - query = searchContext.scanContext().wrapQuery(query); - final ScanCollector scanCollector = searchContext.scanContext().collector(searchContext); - collector = scanCollector; - topDocsCallable = new Callable() { - @Override - public TopDocs call() throws Exception { - return scanCollector.topDocs(); - } - }; } else { // Perhaps have a dedicated scroll phase? final ScrollContext scrollContext = searchContext.scrollContext(); diff --git a/core/src/main/java/org/elasticsearch/search/scan/ScanContext.java b/core/src/main/java/org/elasticsearch/search/scan/ScanContext.java deleted file mode 100644 index b09a81bdab9..00000000000 --- a/core/src/main/java/org/elasticsearch/search/scan/ScanContext.java +++ /dev/null @@ -1,116 +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.scan; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.MinDocQuery; -import org.apache.lucene.search.CollectionTerminatedException; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.SimpleCollector; -import org.apache.lucene.search.TopDocs; -import org.elasticsearch.common.lucene.search.Queries; -import org.elasticsearch.search.internal.SearchContext; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * The scan context allows to optimize readers we already processed during scanning. We do that by keeping track - * of the last collected doc ID and only collecting doc IDs that are greater. - */ -public class ScanContext { - - private volatile int docUpTo; - - public ScanCollector collector(SearchContext context) { - return collector(context.size(), context.trackScores()); - } - - /** Create a {@link ScanCollector} for the given page size. */ - ScanCollector collector(int size, boolean trackScores) { - return new ScanCollector(size, trackScores); - } - - /** - * Wrap the query so that it can skip directly to the right document. - */ - public Query wrapQuery(Query query) { - return Queries.filtered(query, new MinDocQuery(docUpTo)); - } - - public class ScanCollector extends SimpleCollector { - - private final List docs; - - private final int size; - - private final boolean trackScores; - - private Scorer scorer; - - private int docBase; - - private ScanCollector(int size, boolean trackScores) { - this.trackScores = trackScores; - this.docs = new ArrayList<>(size); - this.size = size; - } - - public TopDocs topDocs() { - return new TopDocs(docs.size(), docs.toArray(new ScoreDoc[docs.size()]), 0f); - } - - @Override - public boolean needsScores() { - return trackScores; - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; - } - - @Override - public void collect(int doc) throws IOException { - int topLevelDoc = docBase + doc; - docs.add(new ScoreDoc(topLevelDoc, trackScores ? scorer.score() : 0f)); - // record that we collected up to this document - assert topLevelDoc >= docUpTo; - docUpTo = topLevelDoc + 1; - if (docs.size() >= size) { - throw new CollectionTerminatedException(); - } - } - - @Override - public void doSetNextReader(LeafReaderContext context) throws IOException { - if (docs.size() >= size || context.docBase + context.reader().maxDoc() <= docUpTo) { - // no need to collect a new segment, we either already collected enough - // or the segment is not competitive - throw new CollectionTerminatedException(); - } - docBase = context.docBase; - } - } - -} diff --git a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java index 65349bde389..30dd587380f 100644 --- a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java @@ -88,7 +88,6 @@ import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; @@ -696,27 +695,6 @@ public class IndicesRequestIT extends ESIntegTestCase { assertSameIndicesOptionalRequests(searchRequest, SearchServiceTransportAction.FREE_CONTEXT_ACTION_NAME); } - @Test - public void testSearchScan() throws Exception { - interceptTransportActions(SearchServiceTransportAction.SCAN_ACTION_NAME); - - String[] randomIndicesOrAliases = randomIndicesOrAliases(); - for (int i = 0; i < randomIndicesOrAliases.length; i++) { - client().prepareIndex(randomIndicesOrAliases[i], "type", "id-" + i).setSource("field", "value").get(); - } - refresh(); - - SearchRequest searchRequest = new SearchRequest(randomIndicesOrAliases).searchType(SearchType.SCAN).scroll(new TimeValue(500)); - SearchResponse searchResponse = internalCluster().clientNodeClient().search(searchRequest).actionGet(); - assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().totalHits(), greaterThan(0l)); - - client().prepareClearScroll().addScrollId(searchResponse.getScrollId()).get(); - - clearInterceptedActions(); - assertSameIndices(searchRequest, SearchServiceTransportAction.SCAN_ACTION_NAME); - } - private static void assertSameIndices(IndicesRequest originalRequest, String... actions) { assertSameIndices(originalRequest, false, actions); } diff --git a/core/src/test/java/org/elasticsearch/bwcompat/BasicBackwardsCompatibilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/BasicBackwardsCompatibilityIT.java index 8faccc5c885..bae66293acd 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/BasicBackwardsCompatibilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/BasicBackwardsCompatibilityIT.java @@ -36,7 +36,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.action.update.UpdateRequestBuilder; @@ -680,21 +679,13 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase { int size = randomIntBetween(1, 10); SearchRequestBuilder searchRequestBuilder = client().prepareSearch("test").setScroll("1m").setSize(size); - boolean scan = randomBoolean(); - if (scan) { - searchRequestBuilder.setSearchType(SearchType.SCAN); - } SearchResponse searchResponse = searchRequestBuilder.get(); assertThat(searchResponse.getScrollId(), notNullValue()); assertHitCount(searchResponse, numDocs); int hits = 0; - if (scan) { - assertThat(searchResponse.getHits().getHits().length, equalTo(0)); - } else { - assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); - hits += searchResponse.getHits().getHits().length; - } + assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); + hits += searchResponse.getHits().getHits().length; try { do { diff --git a/core/src/test/java/org/elasticsearch/search/CountSearchIT.java b/core/src/test/java/org/elasticsearch/search/CountSearchIT.java deleted file mode 100644 index 61d887da366..00000000000 --- a/core/src/test/java/org/elasticsearch/search/CountSearchIT.java +++ /dev/null @@ -1,65 +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; - -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.ESIntegTestCase; - -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 CountSearchIT extends ESIntegTestCase { - - 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(); - } - -} 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 e395b0ad528..bdecb756126 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationsIntegrationIT.java @@ -20,9 +20,7 @@ package org.elasticsearch.search.aggregations; 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.common.unit.TimeValue; import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.test.ESIntegTestCase; @@ -52,15 +50,6 @@ public class AggregationsIntegrationIT extends ESIntegTestCase { indexRandom(true, docs); } - public void testScan() { - try { - client().prepareSearch("index").setSearchType(SearchType.SCAN).setScroll(new TimeValue(500)).addAggregation(terms("f").field("f")).get(); - fail(); - } catch (SearchPhaseExecutionException e) { - assertTrue(e.toString(), e.toString().contains("aggregations are not supported with search_type=scan")); - } - } - public void testScroll() { final int size = randomIntBetween(1, 4); SearchResponse response = client().prepareSearch("index") 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 ec9fb42a461..66a9e9f1c51 100644 --- a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java @@ -1439,7 +1439,6 @@ public class ChildQuerySearchIT extends ESIntegTestCase { .setSize(1) .addField("_id") .setQuery(query) - .setSearchType("scan") .execute() .actionGet(); @@ -1447,12 +1446,13 @@ public class ChildQuerySearchIT extends ESIntegTestCase { assertThat(scrollResponse.getHits().totalHits(), equalTo(10l)); int scannedDocs = 0; do { + assertThat(scrollResponse.getHits().totalHits(), equalTo(10l)); + scannedDocs += scrollResponse.getHits().getHits().length; scrollResponse = client() .prepareSearchScroll(scrollResponse.getScrollId()) .setScroll(TimeValue.timeValueSeconds(30)).get(); - assertThat(scrollResponse.getHits().totalHits(), equalTo(10l)); - scannedDocs += scrollResponse.getHits().getHits().length; } while (scrollResponse.getHits().getHits().length > 0); + clearScroll(scrollResponse.getScrollId()); assertThat(scannedDocs, equalTo(10)); } } diff --git a/core/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java b/core/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java deleted file mode 100644 index 38c01cb29e1..00000000000 --- a/core/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java +++ /dev/null @@ -1,96 +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.scan; - -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field.Store; -import org.apache.lucene.document.StringField; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.store.Directory; -import org.elasticsearch.search.scan.ScanContext.ScanCollector; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class ScanContextTests extends ESTestCase { - - private static TopDocs execute(IndexSearcher searcher, ScanContext ctx, Query query, int pageSize, boolean trackScores) throws IOException { - query = ctx.wrapQuery(query); - ScanCollector collector = ctx.collector(pageSize, trackScores); - searcher.search(query, collector); - return collector.topDocs(); - } - - public void testRandom() throws Exception { - final int numDocs = randomIntBetween(10, 200); - final Document doc1 = new Document(); - doc1.add(new StringField("foo", "bar", Store.NO)); - final Document doc2 = new Document(); - final Directory dir = newDirectory(); - final RandomIndexWriter w = new RandomIndexWriter(getRandom(), dir); - for (int i = 0; i < numDocs; ++i) { - w.addDocument(randomBoolean() ? doc1 : doc2); - } - final IndexReader reader = w.getReader(); - final IndexSearcher searcher = newSearcher(reader); - - final boolean trackScores = randomBoolean(); - final int pageSize = randomIntBetween(1, numDocs / 2); - Query query = new TermQuery(new Term("foo", "bar")); - if (trackScores == false) { - query.setBoost(0f); - } - final ScoreDoc[] expected = searcher.search(query, numDocs, Sort.INDEXORDER, true, true).scoreDocs; - - final List actual = new ArrayList<>(); - ScanContext context = new ScanContext(); - while (true) { - final ScoreDoc[] page = execute(searcher,context, query, pageSize, trackScores).scoreDocs; - assertTrue(page.length <= pageSize); - if (page.length == 0) { - assertEquals(0, execute(searcher, context, query, pageSize, trackScores).scoreDocs.length); - break; - } - actual.addAll(Arrays.asList(page)); - } - assertEquals(expected.length, actual.size()); - for (int i = 0; i < expected.length; ++i) { - ScoreDoc sd1 = expected[i]; - ScoreDoc sd2 = actual.get(i); - assertEquals(sd1.doc, sd2.doc); - assertEquals(sd1.score, sd2.score, 0.001f); - } - w.close(); - reader.close(); - dir.close(); - } - -} diff --git a/core/src/test/java/org/elasticsearch/search/scan/SearchScanIT.java b/core/src/test/java/org/elasticsearch/search/scan/SearchScanIT.java deleted file mode 100644 index e98a3c4d80d..00000000000 --- a/core/src/test/java/org/elasticsearch/search/scan/SearchScanIT.java +++ /dev/null @@ -1,85 +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.scan; - -import com.google.common.collect.Sets; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.test.ESIntegTestCase; -import org.junit.Test; - -import java.util.Set; - -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; -import static org.hamcrest.Matchers.equalTo; - -public class SearchScanIT extends ESIntegTestCase { - - @Test - public void testNarrowingQuery() throws Exception { - createIndex("test"); - ensureGreen(); - - Set ids = Sets.newHashSet(); - Set expectedIds = Sets.newHashSet(); - IndexRequestBuilder[] builders = new IndexRequestBuilder[scaledRandomIntBetween(50, 100)]; - for (int i = 0; i < builders.length/2; i++) { - expectedIds.add(Integer.toString(i)); - builders[i] = client().prepareIndex("test", "tweet", Integer.toString(i)).setSource( - jsonBuilder().startObject().field("user", "kimchy1").field("postDate", System.currentTimeMillis()).field("message", "test").endObject()); - } - - for (int i = builders.length/2; i < builders.length; i++) { - builders[i] = client().prepareIndex("test", "tweet", Integer.toString(i)).setSource( - jsonBuilder().startObject().field("user", "kimchy2").field("postDate", System.currentTimeMillis()).field("message", "test").endObject()); - } - indexRandom(true, builders); - - SearchResponse searchResponse = client().prepareSearch() - .setSearchType(SearchType.SCAN) - .setQuery(termQuery("user", "kimchy1")) - .setSize(35) - .setScroll(TimeValue.timeValueMinutes(2)) - .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) { - searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(2)).execute().actionGet(); - assertThat(searchResponse.getHits().totalHits(), equalTo((long)builders.length/2)); - assertThat(searchResponse.getFailedShards(), equalTo(0)); - for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id() + "should not exists in the result set", ids.contains(hit.id()), equalTo(false)); - ids.add(hit.id()); - } - if (searchResponse.getHits().hits().length == 0) { - break; - } - } - - assertThat(expectedIds, equalTo(ids)); - } -} \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/search/scan/SearchScanScrollingIT.java b/core/src/test/java/org/elasticsearch/search/scan/SearchScanScrollingIT.java deleted file mode 100644 index 8c7a31140bb..00000000000 --- a/core/src/test/java/org/elasticsearch/search/scan/SearchScanScrollingIT.java +++ /dev/null @@ -1,106 +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.scan; - -import com.google.common.collect.Sets; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.test.ESIntegTestCase; -import org.junit.Test; - -import java.util.Set; - -import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; - -public class SearchScanScrollingIT extends ESIntegTestCase { - - @Test - public void testRandomized() throws Exception { - testScroll(scaledRandomIntBetween(100, 200), between(1, 300), getRandom().nextBoolean(), getRandom().nextBoolean()); - } - - private void testScroll(long numberOfDocs, int size, boolean unbalanced, boolean trackScores) throws Exception { - createIndex("test"); - ensureGreen(); - - Set ids = Sets.newHashSet(); - Set expectedIds = Sets.newHashSet(); - for (int i = 0; i < numberOfDocs; i++) { - String id = Integer.toString(i); - expectedIds.add(id); - String routing = null; - if (unbalanced) { - if (i < (numberOfDocs * 0.6)) { - routing = "0"; - } else if (i < (numberOfDocs * 0.9)) { - routing = "1"; - } else { - routing = "2"; - } - } - client().prepareIndex("test", "type1", id).setRouting(routing).setSource("field", i).execute().actionGet(); - // make some segments - if (i % 10 == 0) { - client().admin().indices().prepareFlush().execute().actionGet(); - } - } - - refresh(); - - SearchResponse searchResponse = client().prepareSearch() - .setSearchType(SearchType.SCAN) - .setQuery(matchAllQuery()) - .setSize(size) - .setScroll(TimeValue.timeValueMinutes(2)) - .setTrackScores(trackScores) - .execute().actionGet(); - try { - assertHitCount(searchResponse, numberOfDocs); - - // start scrolling, until we get not results - while (true) { - searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(2)).execute().actionGet(); - assertHitCount(searchResponse, numberOfDocs); - - for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.id() + "should not exist in the result set", ids.contains(hit.id()), equalTo(false)); - ids.add(hit.id()); - if (trackScores) { - assertThat(hit.getScore(), greaterThan(0.0f)); - } else { - assertThat(hit.getScore(), equalTo(0.0f)); - } - } - if (searchResponse.getHits().hits().length == 0) { - break; - } - } - - assertThat(expectedIds, equalTo(ids)); - } finally { - clearScroll(searchResponse.getScrollId()); - } - } -} \ No newline at end of file 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 54f6f14e514..1ac4d43b140 100644 --- a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java +++ b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java @@ -424,7 +424,7 @@ public class SearchScrollIT extends ESIntegTestCase { .setQuery(QueryBuilders.matchAllQuery()) .setSize(Integer.MAX_VALUE); - if (searchType == SearchType.SCAN || searchType != SearchType.COUNT && randomBoolean()) { + if (randomBoolean()) { builder.setScroll("1m"); } diff --git a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java index 3a6af88a683..be9bfde027f 100644 --- a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java +++ b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardIterator; @@ -185,7 +184,6 @@ public class SearchStatsIT extends ESIntegTestCase { int size = scaledRandomIntBetween(1, docs); SearchResponse searchResponse = client().prepareSearch() - .setSearchType(SearchType.SCAN) .setQuery(matchAllQuery()) .setSize(size) .setScroll(TimeValue.timeValueMinutes(2)) @@ -200,33 +198,33 @@ public class SearchStatsIT extends ESIntegTestCase { int hits = 0; while (true) { + if (searchResponse.getHits().getHits().length == 0) { + break; + } hits += searchResponse.getHits().getHits().length; searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()) .setScroll(TimeValue.timeValueMinutes(2)) .execute().actionGet(); - if (searchResponse.getHits().getHits().length == 0) { - break; - } } long expected = 0; - // the number of queries executed is equal to the sum of 1 + number of pages in shard over all shards + // the number of queries executed is equal to at least the sum of number of pages in shard over all shards IndicesStatsResponse r = client().admin().indices().prepareStats(index).execute().actionGet(); for (int s = 0; s < numAssignedShards(index); s++) { - expected += 1 + (long)Math.ceil(r.getShards()[s].getStats().getDocs().getCount() / size); + expected += (long)Math.ceil(r.getShards()[s].getStats().getDocs().getCount() / size); } indicesStats = client().admin().indices().prepareStats().execute().actionGet(); Stats stats = indicesStats.getTotal().getSearch().getTotal(); assertEquals(hits, docs * numAssignedShards(index)); - assertThat(stats.getQueryCount(), equalTo(expected)); - assertThat(stats.getScrollCount(), equalTo((long)numAssignedShards(index))); - assertThat(stats.getScrollTimeInMillis(), greaterThan(0l)); + assertThat(stats.getQueryCount(), greaterThanOrEqualTo(expected)); - // scroll, but with no timeout (so no context) - searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).execute().actionGet(); + clearScroll(searchResponse.getScrollId()); indicesStats = client().admin().indices().prepareStats().execute().actionGet(); + stats = indicesStats.getTotal().getSearch().getTotal(); assertThat(indicesStats.getTotal().getSearch().getOpenContexts(), equalTo(0l)); + assertThat(stats.getScrollCount(), equalTo((long)numAssignedShards(index))); + assertThat(stats.getScrollTimeInMillis(), greaterThan(0l)); } protected int numAssignedShards(String... indices) { diff --git a/core/src/test/java/org/elasticsearch/test/TestSearchContext.java b/core/src/test/java/org/elasticsearch/test/TestSearchContext.java index 56766b748b0..a616ab81082 100644 --- a/core/src/test/java/org/elasticsearch/test/TestSearchContext.java +++ b/core/src/test/java/org/elasticsearch/test/TestSearchContext.java @@ -63,7 +63,6 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; -import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import org.elasticsearch.threadpool.ThreadPool; @@ -555,11 +554,6 @@ public class TestSearchContext extends SearchContext { return null; } - @Override - public ScanContext scanContext() { - return null; - } - @Override public MappedFieldType smartNameFieldType(String name) { if (mapperService() != null) { diff --git a/docs/reference/migration/index.asciidoc b/docs/reference/migration/index.asciidoc index 19a3e1f622b..1a26fe81a20 100644 --- a/docs/reference/migration/index.asciidoc +++ b/docs/reference/migration/index.asciidoc @@ -16,6 +16,8 @@ As a general rule: See <> for more info. -- +include::migrate_3_0.asciidoc[] + include::migrate_2_1.asciidoc[] include::migrate_2_0.asciidoc[] diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc new file mode 100644 index 00000000000..55f0ad4ee5f --- /dev/null +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -0,0 +1,65 @@ +[[breaking-changes-3.0]] +== Breaking changes in 3.0 + +This section discusses the changes that you need to be aware of when migrating +your application to Elasticsearch 3.0. + +=== Search changes + +==== `search_type=count` removed + +The `count` search type was deprecated since version 2.0.0 and is now removed. +In order to get the same benefits, you just need to set the value of the `size` +parameter to `0`. + +For instance, the following request: +[source,sh] +--------------- +GET /my_index/_search?search_type=count +{ + "aggs": { + "my_terms": { + "terms": { + "field": "foo" + } + } + } +} +--------------- + +can be replaced with: +[source,sh] +--------------- +GET /my_index/_search +{ + "size": 0, + "aggs": { + "my_terms": { + "terms": { + "field": "foo" + } + } + } +} +--------------- + +==== `search_type=scan` removed + +The `scan` search type was deprecated since version 2.1.0 and is now removed. +All benefits from this search type can now be achieved by doing a scroll +request that sorts documents in `_doc` order, for instance: + +[source,sh] +--------------- +GET /my_index/_search?scroll=2m +{ + "sort": [ + "_doc" + ] +} +--------------- + +Scroll requests sorted by `_doc` have been optimized to more efficiently resume +from where the previous request stopped, so this will have the same performance +characteristics as the former `scan` search type. + diff --git a/docs/reference/search/request/rescore.asciidoc b/docs/reference/search/request/rescore.asciidoc index 729ec8ad839..8bab1cf1f5c 100644 --- a/docs/reference/search/request/rescore.asciidoc +++ b/docs/reference/search/request/rescore.asciidoc @@ -15,10 +15,6 @@ Currently the rescore API has only one implementation: the query rescorer, which uses a query to tweak the scoring. In the future, alternative rescorers may be made available, for example, a pair-wise rescorer. -NOTE: the `rescore` phase is not executed when -<> is set -to `scan` or `count`. - NOTE: when exposing pagination to your users, you should not change `window_size` as you step through each page (by passing different `from` values) since that can alter the top hits causing results to diff --git a/docs/reference/search/request/search-type.asciidoc b/docs/reference/search/request/search-type.asciidoc index ab2d9164988..622b01c453e 100644 --- a/docs/reference/search/request/search-type.asciidoc +++ b/docs/reference/search/request/search-type.asciidoc @@ -62,26 +62,4 @@ Same as "Query Then Fetch", except for an initial scatter phase which goes and computes the distributed term frequencies for more accurate scoring. -[[count]] -==== Count - -deprecated[2.0.0-beta1, `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 -request without any docs (represented in `total_hits`), and possibly, -including aggregations as well. In general, this is preferable to the `count` -API as it provides more options. - -[[scan]] -==== Scan - -deprecated[2.1.0, `scan` does not provide any benefits over a regular `scroll` request sorted by `_doc`] - -Parameter value: *scan*. - -The `scan` search type disables sorting in order to allow very efficient -scrolling through large result sets. - diff --git a/docs/reference/search/uri-request.asciidoc b/docs/reference/search/uri-request.asciidoc index e01f9e9d2c7..c2322e53cf9 100644 --- a/docs/reference/search/uri-request.asciidoc +++ b/docs/reference/search/uri-request.asciidoc @@ -103,10 +103,8 @@ Defaults to no terminate_after. |`size` |The number of hits to return. Defaults to `10`. |`search_type` |The type of the search operation to perform. Can be -`dfs_query_then_fetch`, `query_then_fetch`, `scan` -deprecated[2.1.0,Replaced by a regular `scroll` sorted by `_doc`] -or `count` -deprecated[2.0.0-beta1,Replaced by `size: 0`]. Defaults to `query_then_fetch`. See +`dfs_query_then_fetch` or `query_then_fetch`. +Defaults to `query_then_fetch`. See <> for more details on the different types of search that can be performed. |======================================================================= diff --git a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java index 4ba9d09edaa..05f015b279c 100644 --- a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java +++ b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.Strings; import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.unit.TimeValue; @@ -43,7 +42,7 @@ import org.junit.Test; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.rangeQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.*; public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { @@ -131,7 +130,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { client().admin().indices().prepareRefresh("test").get(); assertHitCount(client().prepareCount("test").get(), numDocs); - SearchResponse searchResponse = client().prepareSearch("test").setSearchType(SearchType.SCAN).setScroll(TimeValue.timeValueSeconds(10)).get(); + SearchResponse searchResponse = client().prepareSearch("test").setScroll(TimeValue.timeValueSeconds(10)).get(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocs)); String scrollId = searchResponse.getScrollId(); @@ -146,16 +145,18 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { newAsyncAction(delete, listener).executeScroll(searchResponse.getScrollId()); waitForCompletion("scroll request returns zero documents on expired scroll id", listener); - assertNull(listener.getError()); - assertShardFailuresContains(listener.getResponse().getShardFailures(), "No search context found"); + assertNotNull(listener.getError()); + assertThrowableContains(listener.getError(), "No search context found"); assertSearchContextsClosed(); } @Test public void testExecuteScrollTimedOut() throws InterruptedException { - client().prepareIndex("test", "type").setSource("num", "1").setRefresh(true).get(); + client().prepareIndex("test", "type", "1").setSource("num", "1").get(); + client().prepareIndex("test", "type", "2").setSource("num", "1").get(); + client().admin().indices().prepareRefresh("test").get(); - SearchResponse searchResponse = client().prepareSearch("test").setSearchType(SearchType.SCAN).setScroll(TimeValue.timeValueSeconds(10)).get(); + SearchResponse searchResponse = client().prepareSearch("test").setSize(1).setScroll(TimeValue.timeValueSeconds(10)).get(); String scrollId = searchResponse.getScrollId(); assertTrue(Strings.hasText(scrollId)); @@ -178,7 +179,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { @Test public void testExecuteScrollNoDocuments() { createIndex("test"); - SearchResponse searchResponse = client().prepareSearch("test").setSearchType(SearchType.SCAN).setScroll(TimeValue.timeValueSeconds(10)).get(); + SearchResponse searchResponse = client().prepareSearch("test").setScroll(TimeValue.timeValueSeconds(10)).get(); String scrollId = searchResponse.getScrollId(); assertTrue(Strings.hasText(scrollId)); @@ -206,7 +207,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { final long limit = randomIntBetween(0, numDocs); - SearchResponse searchResponse = client().prepareSearch("test").setSearchType(SearchType.SCAN) + SearchResponse searchResponse = client().prepareSearch("test") .setScroll(TimeValue.timeValueSeconds(10)) .setQuery(boolQuery().must(rangeQuery("num").lte(limit))) .addFields("_routing", "_parent") @@ -226,7 +227,9 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { assertNull(listener.getError()); assertFalse(listener.getResponse().isTimedOut()); - assertThat(listener.getResponse().getTotalDeleted(), equalTo(limit)); + // docs that have been returned on the 1st page have been skipped + final long expectedDeleted = Math.max(0, limit - searchResponse.getHits().hits().length); + assertThat(listener.getResponse().getTotalDeleted(), equalTo(expectedDeleted)); assertSearchContextsClosed(); } @@ -256,7 +259,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { // We just need a valid scroll id createIndex("test"); - SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.SCAN).setScroll(TimeValue.timeValueSeconds(10)).get(); + SearchResponse searchResponse = client().prepareSearch().setScroll(TimeValue.timeValueSeconds(10)).get(); String scrollId = searchResponse.getScrollId(); assertTrue(Strings.hasText(scrollId)); @@ -314,7 +317,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { // We just need a valid scroll id createIndex("test"); - SearchResponse searchResponse = client().prepareSearch().setSearchType(SearchType.SCAN).setScroll(TimeValue.timeValueSeconds(10)).get(); + SearchResponse searchResponse = client().prepareSearch().setScroll(TimeValue.timeValueSeconds(10)).get(); String scrollId = searchResponse.getScrollId(); assertTrue(Strings.hasText(scrollId)); @@ -439,10 +442,15 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase { } } + private void assertThrowableContains(Throwable t, String expectedFailure) { + assertThat(t.toString(), containsString(expectedFailure)); + } + private void assertShardFailuresContains(ShardOperationFailedException[] shardFailures, String expectedFailure) { assertNotNull(shardFailures); for (ShardOperationFailedException failure : shardFailures) { - if (failure.reason().contains(expectedFailure)) { + Throwable t = failure.getCause(); + if (t.toString().contains(expectedFailure)) { return; } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json index 3d8297e496d..87a86fb1298 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json @@ -18,7 +18,7 @@ "params": { "search_type": { "type" : "enum", - "options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch", "count", "scan"], + "options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch"], "description" : "Search operation type" } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index f0ca4d80baf..d2b9b8cf9b4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -90,7 +90,7 @@ }, "search_type": { "type" : "enum", - "options" : ["query_then_fetch", "dfs_query_then_fetch", "count", "scan"], + "options" : ["query_then_fetch", "dfs_query_then_fetch"], "description" : "Search operation type" }, "size": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json index a1122f19a1e..67cbcf44a9e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json @@ -44,7 +44,7 @@ }, "search_type": { "type" : "enum", - "options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch", "count", "scan"], + "options" : ["query_then_fetch", "query_and_fetch", "dfs_query_then_fetch", "dfs_query_and_fetch"], "description" : "Search operation type" } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic.yaml index d8205f66881..4013f315d46 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic.yaml @@ -1,68 +1,3 @@ ---- -"Basic scan": - - do: - indices.create: - index: test_scroll - body: - settings: - number_of_shards: "1" - - do: - index: - index: test_scroll - type: test - id: 42 - body: { foo: bar } - - - do: - index: - index: test_scroll - type: test - id: 43 - body: { foo: bar } - - - do: - indices.refresh: {} - - - do: - search: - index: test_scroll - search_type: scan - size: 1 - scroll: 1m - body: - query: - match_all: {} - - - set: {_scroll_id: scroll_id} - - match: {hits.total: 2 } - - length: {hits.hits: 0 } - - - do: - scroll: - scroll_id: $scroll_id - scroll: 1m - - - set: {_scroll_id: scroll_id} - - match: {hits.total: 2 } - - length: {hits.hits: 1 } - - - do: - scroll: - scroll_id: $scroll_id - scroll: 1m - - - set: {_scroll_id: scroll_id} - - match: {hits.total: 2 } - - length: {hits.hits: 1 } - - - do: - scroll: - scroll_id: $scroll_id - scroll: 1m - - - match: {hits.total: 2 } - - length: {hits.hits: 0 } - --- "Basic scroll": - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/11_clear.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/11_clear.yaml index b620869c88f..aa4885825d2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/11_clear.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/11_clear.yaml @@ -16,7 +16,6 @@ - do: search: index: test_scroll - search_type: scan scroll: 1m body: query: @@ -56,7 +55,6 @@ - do: search: index: test_scroll - search_type: scan scroll: 1m body: query: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_search_count.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_search_count.yaml deleted file mode 100644 index 73a3ee96b5a..00000000000 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_search_count.yaml +++ /dev/null @@ -1,33 +0,0 @@ ---- -"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}