/* * Licensed to ElasticSearch and Shay Banon 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.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.*; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.action.SearchServiceListener; import org.elasticsearch.search.action.SearchServiceTransportAction; import org.elasticsearch.search.controller.SearchPhaseController; import org.elasticsearch.search.controller.ShardDoc; import org.elasticsearch.search.fetch.QueryFetchSearchResult; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.threadpool.ThreadPool; import java.util.Map; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest; /** * */ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent { private final ThreadPool threadPool; private final ClusterService clusterService; private final SearchServiceTransportAction searchService; private final SearchPhaseController searchPhaseController; private final TransportSearchCache searchCache; @Inject public TransportSearchScrollQueryAndFetchAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportSearchCache searchCache, SearchServiceTransportAction searchService, SearchPhaseController searchPhaseController) { super(settings); this.threadPool = threadPool; this.clusterService = clusterService; this.searchCache = searchCache; this.searchService = searchService; this.searchPhaseController = searchPhaseController; } public void execute(SearchScrollRequest request, ParsedScrollId scrollId, ActionListener<SearchResponse> listener) { new AsyncAction(request, scrollId, listener).start(); } private class AsyncAction { private final SearchScrollRequest request; private final ActionListener<SearchResponse> listener; private final ParsedScrollId scrollId; private final DiscoveryNodes nodes; private volatile Queue<ShardSearchFailure> shardFailures; private final Map<SearchShardTarget, QueryFetchSearchResult> queryFetchResults = searchCache.obtainQueryFetchResults(); private final AtomicInteger successfulOps; private final AtomicInteger counter; private final long startTime = System.currentTimeMillis(); private AsyncAction(SearchScrollRequest request, ParsedScrollId scrollId, ActionListener<SearchResponse> listener) { this.request = request; this.listener = listener; this.scrollId = scrollId; this.nodes = clusterService.state().nodes(); this.successfulOps = new AtomicInteger(scrollId.context().length); this.counter = new AtomicInteger(scrollId.context().length); } protected final ShardSearchFailure[] buildShardFailures() { Queue<ShardSearchFailure> localFailures = shardFailures; if (localFailures == null) { return ShardSearchFailure.EMPTY_ARRAY; } return localFailures.toArray(ShardSearchFailure.EMPTY_ARRAY); } // 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(ShardSearchFailure failure) { if (shardFailures == null) { shardFailures = ConcurrentCollections.newQueue(); } shardFailures.add(failure); } public void start() { if (scrollId.context().length == 0) { listener.onFailure(new SearchPhaseExecutionException("query", "no nodes to search on", null)); return; } int localOperations = 0; for (Tuple<String, Long> target : scrollId.context()) { DiscoveryNode node = nodes.get(target.v1()); if (node != null) { if (nodes.localNodeId().equals(node.id())) { localOperations++; } else { executePhase(node, target.v2()); } } else { if (logger.isDebugEnabled()) { logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.source() + "]"); } successfulOps.decrementAndGet(); if (counter.decrementAndGet() == 0) { finishHim(); } } } if (localOperations > 0) { if (request.operationThreading() == SearchOperationThreading.SINGLE_THREAD) { threadPool.executor(ThreadPool.Names.SEARCH).execute(new Runnable() { @Override public void run() { for (Tuple<String, Long> target : scrollId.context()) { DiscoveryNode node = nodes.get(target.v1()); if (node != null && nodes.localNodeId().equals(node.id())) { executePhase(node, target.v2()); } } } }); } else { boolean localAsync = request.operationThreading() == SearchOperationThreading.THREAD_PER_SHARD; for (final Tuple<String, Long> target : scrollId.context()) { final DiscoveryNode node = nodes.get(target.v1()); if (node != null && nodes.localNodeId().equals(node.id())) { if (localAsync) { threadPool.executor(ThreadPool.Names.SEARCH).execute(new Runnable() { @Override public void run() { executePhase(node, target.v2()); } }); } else { executePhase(node, target.v2()); } } } } } for (Tuple<String, Long> target : scrollId.context()) { DiscoveryNode node = nodes.get(target.v1()); if (node == null) { if (logger.isDebugEnabled()) { logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.source() + "]"); } successfulOps.decrementAndGet(); if (counter.decrementAndGet() == 0) { finishHim(); } } else { } } } private void executePhase(DiscoveryNode node, final long searchId) { searchService.sendExecuteFetch(node, internalScrollSearchRequest(searchId, request), new SearchServiceListener<QueryFetchSearchResult>() { @Override public void onResult(QueryFetchSearchResult result) { queryFetchResults.put(result.shardTarget(), result); if (counter.decrementAndGet() == 0) { finishHim(); } } @Override public void onFailure(Throwable t) { if (logger.isDebugEnabled()) { logger.debug("[{}] Failed to execute query phase", t, searchId); } addShardFailure(new ShardSearchFailure(t)); successfulOps.decrementAndGet(); if (counter.decrementAndGet() == 0) { finishHim(); } } }); } private void finishHim() { try { innerFinishHim(); } catch (Exception e) { listener.onFailure(new ReduceSearchPhaseException("fetch", "", e, buildShardFailures())); } } private void innerFinishHim() { ShardDoc[] sortedShardList = searchPhaseController.sortDocs(queryFetchResults.values()); final InternalSearchResponse internalResponse = searchPhaseController.merge(sortedShardList, queryFetchResults, queryFetchResults); String scrollId = null; if (request.scroll() != null) { scrollId = request.scrollId(); } searchCache.releaseQueryFetchResults(queryFetchResults); listener.onResponse(new SearchResponse(internalResponse, scrollId, this.scrollId.context().length, successfulOps.get(), System.currentTimeMillis() - startTime, buildShardFailures())); } } }