|  | @@ -19,74 +19,67 @@
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  package org.elasticsearch.action.search;
 |  |  package org.elasticsearch.action.search;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -import com.carrotsearch.hppc.IntArrayList;
 |  | 
 | 
											
												
													
														|  |  import org.apache.logging.log4j.Logger;
 |  |  import org.apache.logging.log4j.Logger;
 | 
											
												
													
														|  |  import org.apache.logging.log4j.message.ParameterizedMessage;
 |  |  import org.apache.logging.log4j.message.ParameterizedMessage;
 | 
											
												
													
														|  |  import org.apache.logging.log4j.util.Supplier;
 |  |  import org.apache.logging.log4j.util.Supplier;
 | 
											
												
													
														|  | -import org.apache.lucene.search.ScoreDoc;
 |  | 
 | 
											
												
													
														|  | 
 |  | +import org.apache.lucene.util.SetOnce;
 | 
											
												
													
														|  | 
 |  | +import org.elasticsearch.ElasticsearchException;
 | 
											
												
													
														|  | 
 |  | +import org.elasticsearch.ExceptionsHelper;
 | 
											
												
													
														|  |  import org.elasticsearch.action.ActionListener;
 |  |  import org.elasticsearch.action.ActionListener;
 | 
											
												
													
														|  | -import org.elasticsearch.action.ActionRunnable;
 |  | 
 | 
											
												
													
														|  | -import org.elasticsearch.action.NoShardAvailableActionException;
 |  | 
 | 
											
												
													
														|  | 
 |  | +import org.elasticsearch.action.ShardOperationFailedException;
 | 
											
												
													
														|  |  import org.elasticsearch.action.support.TransportActions;
 |  |  import org.elasticsearch.action.support.TransportActions;
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.routing.GroupShardsIterator;
 |  |  import org.elasticsearch.cluster.routing.GroupShardsIterator;
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.routing.ShardIterator;
 |  |  import org.elasticsearch.cluster.routing.ShardIterator;
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.routing.ShardRouting;
 |  |  import org.elasticsearch.cluster.routing.ShardRouting;
 | 
											
												
													
														|  | -import org.elasticsearch.common.CheckedRunnable;
 |  | 
 | 
											
												
													
														|  |  import org.elasticsearch.common.Nullable;
 |  |  import org.elasticsearch.common.Nullable;
 | 
											
												
													
														|  |  import org.elasticsearch.common.util.concurrent.AtomicArray;
 |  |  import org.elasticsearch.common.util.concurrent.AtomicArray;
 | 
											
												
													
														|  | -import org.elasticsearch.common.util.concurrent.CountDown;
 |  | 
 | 
											
												
													
														|  |  import org.elasticsearch.search.SearchPhaseResult;
 |  |  import org.elasticsearch.search.SearchPhaseResult;
 | 
											
												
													
														|  |  import org.elasticsearch.search.SearchShardTarget;
 |  |  import org.elasticsearch.search.SearchShardTarget;
 | 
											
												
													
														|  | -import org.elasticsearch.search.fetch.FetchSearchResult;
 |  | 
 | 
											
												
													
														|  | -import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
 |  | 
 | 
											
												
													
														|  |  import org.elasticsearch.search.internal.AliasFilter;
 |  |  import org.elasticsearch.search.internal.AliasFilter;
 | 
											
												
													
														|  |  import org.elasticsearch.search.internal.InternalSearchResponse;
 |  |  import org.elasticsearch.search.internal.InternalSearchResponse;
 | 
											
												
													
														|  |  import org.elasticsearch.search.internal.ShardSearchTransportRequest;
 |  |  import org.elasticsearch.search.internal.ShardSearchTransportRequest;
 | 
											
												
													
														|  | -import org.elasticsearch.search.query.QuerySearchResult;
 |  | 
 | 
											
												
													
														|  | -import org.elasticsearch.search.query.QuerySearchResultProvider;
 |  | 
 | 
											
												
													
														|  | -import org.elasticsearch.transport.ConnectTransportException;
 |  | 
 | 
											
												
													
														|  |  import org.elasticsearch.transport.Transport;
 |  |  import org.elasticsearch.transport.Transport;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -import java.io.IOException;
 |  | 
 | 
											
												
													
														|  |  import java.util.List;
 |  |  import java.util.List;
 | 
											
												
													
														|  |  import java.util.Map;
 |  |  import java.util.Map;
 | 
											
												
													
														|  | 
 |  | +import java.util.StringJoiner;
 | 
											
												
													
														|  |  import java.util.concurrent.Executor;
 |  |  import java.util.concurrent.Executor;
 | 
											
												
													
														|  |  import java.util.concurrent.atomic.AtomicInteger;
 |  |  import java.util.concurrent.atomic.AtomicInteger;
 | 
											
												
													
														|  |  import java.util.function.Function;
 |  |  import java.util.function.Function;
 | 
											
												
													
														|  | -import java.util.function.IntConsumer;
 |  | 
 | 
											
												
													
														|  | 
 |  | +import java.util.stream.Collectors;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult> extends AbstractAsyncAction {
 |  | 
 | 
											
												
													
														|  | 
 |  | +abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> extends InitialSearchPhase<Result>
 | 
											
												
													
														|  | 
 |  | +    implements SearchPhaseContext {
 | 
											
												
													
														|  |      private static final float DEFAULT_INDEX_BOOST = 1.0f;
 |  |      private static final float DEFAULT_INDEX_BOOST = 1.0f;
 | 
											
												
													
														|  | -    protected final Logger logger;
 |  | 
 | 
											
												
													
														|  | -    protected final SearchTransportService searchTransportService;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private final Logger logger;
 | 
											
												
													
														|  | 
 |  | +    private final SearchTransportService searchTransportService;
 | 
											
												
													
														|  |      private final Executor executor;
 |  |      private final Executor executor;
 | 
											
												
													
														|  | -    protected final ActionListener<SearchResponse> listener;
 |  | 
 | 
											
												
													
														|  | -    private final GroupShardsIterator shardsIts;
 |  | 
 | 
											
												
													
														|  | -    protected final SearchRequest request;
 |  | 
 | 
											
												
													
														|  | -    /** Used by subclasses to resolve node ids to DiscoveryNodes. **/
 |  | 
 | 
											
												
													
														|  | -    protected final Function<String, Transport.Connection> nodeIdToConnection;
 |  | 
 | 
											
												
													
														|  | -    protected final SearchPhaseController searchPhaseController;
 |  | 
 | 
											
												
													
														|  | -    protected final SearchTask task;
 |  | 
 | 
											
												
													
														|  | -    private final int expectedSuccessfulOps;
 |  | 
 | 
											
												
													
														|  | -    private final int expectedTotalOps;
 |  | 
 | 
											
												
													
														|  | -    private final AtomicInteger successfulOps = new AtomicInteger();
 |  | 
 | 
											
												
													
														|  | -    private final AtomicInteger totalOps = new AtomicInteger();
 |  | 
 | 
											
												
													
														|  | -    private final AtomicArray<FirstResult> initialResults;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private final ActionListener<SearchResponse> listener;
 | 
											
												
													
														|  | 
 |  | +    private final SearchRequest request;
 | 
											
												
													
														|  | 
 |  | +    /**
 | 
											
												
													
														|  | 
 |  | +     * Used by subclasses to resolve node ids to DiscoveryNodes.
 | 
											
												
													
														|  | 
 |  | +     **/
 | 
											
												
													
														|  | 
 |  | +    private final Function<String, Transport.Connection> nodeIdToConnection;
 | 
											
												
													
														|  | 
 |  | +    private final SearchTask task;
 | 
											
												
													
														|  | 
 |  | +    private final AtomicArray<Result> results;
 | 
											
												
													
														|  | 
 |  | +    private final long clusterStateVersion;
 | 
											
												
													
														|  |      private final Map<String, AliasFilter> aliasFilter;
 |  |      private final Map<String, AliasFilter> aliasFilter;
 | 
											
												
													
														|  |      private final Map<String, Float> concreteIndexBoosts;
 |  |      private final Map<String, Float> concreteIndexBoosts;
 | 
											
												
													
														|  | -    private final long clusterStateVersion;
 |  | 
 | 
											
												
													
														|  | -    private volatile AtomicArray<ShardSearchFailure> shardFailures;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private final SetOnce<AtomicArray<ShardSearchFailure>> shardFailures = new SetOnce<>();
 | 
											
												
													
														|  |      private final Object shardFailuresMutex = new Object();
 |  |      private final Object shardFailuresMutex = new Object();
 | 
											
												
													
														|  | 
 |  | +    private final AtomicInteger successfulOps = new AtomicInteger();
 | 
											
												
													
														|  | 
 |  | +    private final long startTime;
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    protected AbstractSearchAsyncAction(Logger logger, SearchTransportService searchTransportService,
 |  | 
 | 
											
												
													
														|  | 
 |  | +    protected AbstractSearchAsyncAction(String name, Logger logger, SearchTransportService searchTransportService,
 | 
											
												
													
														|  |                                          Function<String, Transport.Connection> nodeIdToConnection,
 |  |                                          Function<String, Transport.Connection> nodeIdToConnection,
 | 
											
												
													
														|  |                                          Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
 |  |                                          Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
 | 
											
												
													
														|  | -                                        SearchPhaseController searchPhaseController, Executor executor, SearchRequest request,
 |  | 
 | 
											
												
													
														|  | 
 |  | +                                        Executor executor, SearchRequest request,
 | 
											
												
													
														|  |                                          ActionListener<SearchResponse> listener, GroupShardsIterator shardsIts, long startTime,
 |  |                                          ActionListener<SearchResponse> listener, GroupShardsIterator shardsIts, long startTime,
 | 
											
												
													
														|  |                                          long clusterStateVersion, SearchTask task) {
 |  |                                          long clusterStateVersion, SearchTask task) {
 | 
											
												
													
														|  | -        super(startTime);
 |  | 
 | 
											
												
													
														|  | 
 |  | +        super(name, request, shardsIts, logger);
 | 
											
												
													
														|  | 
 |  | +        this.startTime = startTime;
 | 
											
												
													
														|  |          this.logger = logger;
 |  |          this.logger = logger;
 | 
											
												
													
														|  | -        this.searchPhaseController = searchPhaseController;
 |  | 
 | 
											
												
													
														|  |          this.searchTransportService = searchTransportService;
 |  |          this.searchTransportService = searchTransportService;
 | 
											
												
													
														|  |          this.executor = executor;
 |  |          this.executor = executor;
 | 
											
												
													
														|  |          this.request = request;
 |  |          this.request = request;
 | 
											
										
											
												
													
														|  | @@ -94,175 +87,75 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
 | 
											
												
													
														|  |          this.listener = listener;
 |  |          this.listener = listener;
 | 
											
												
													
														|  |          this.nodeIdToConnection = nodeIdToConnection;
 |  |          this.nodeIdToConnection = nodeIdToConnection;
 | 
											
												
													
														|  |          this.clusterStateVersion = clusterStateVersion;
 |  |          this.clusterStateVersion = clusterStateVersion;
 | 
											
												
													
														|  | -        this.shardsIts = shardsIts;
 |  | 
 | 
											
												
													
														|  | -        expectedSuccessfulOps = shardsIts.size();
 |  | 
 | 
											
												
													
														|  | -        // we need to add 1 for non active partition, since we count it in the total!
 |  | 
 | 
											
												
													
														|  | -        expectedTotalOps = shardsIts.totalSizeWith1ForEmpty();
 |  | 
 | 
											
												
													
														|  | -        initialResults = new AtomicArray<>(shardsIts.size());
 |  | 
 | 
											
												
													
														|  | -        this.aliasFilter = aliasFilter;
 |  | 
 | 
											
												
													
														|  | 
 |  | +        results = new AtomicArray<>(shardsIts.size());
 | 
											
												
													
														|  |          this.concreteIndexBoosts = concreteIndexBoosts;
 |  |          this.concreteIndexBoosts = concreteIndexBoosts;
 | 
											
												
													
														|  | 
 |  | +        this.aliasFilter = aliasFilter;
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +    /**
 | 
											
												
													
														|  | 
 |  | +     * Builds how long it took to execute the search.
 | 
											
												
													
														|  | 
 |  | +     */
 | 
											
												
													
														|  | 
 |  | +    private long buildTookInMillis() {
 | 
											
												
													
														|  | 
 |  | +        // protect ourselves against time going backwards
 | 
											
												
													
														|  | 
 |  | +        // negative values don't make sense and we want to be able to serialize that thing as a vLong
 | 
											
												
													
														|  | 
 |  | +        return Math.max(1, System.currentTimeMillis() - startTime);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    public void start() {
 |  | 
 | 
											
												
													
														|  | -        if (expectedSuccessfulOps == 0) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +    /**
 | 
											
												
													
														|  | 
 |  | +     * This is the main entry point for a search. This method starts the search execution of the initial phase.
 | 
											
												
													
														|  | 
 |  | +     */
 | 
											
												
													
														|  | 
 |  | +    public final void start() {
 | 
											
												
													
														|  | 
 |  | +        if (results.length() == 0) {
 | 
											
												
													
														|  |              //no search shards to search on, bail with empty response
 |  |              //no search shards to search on, bail with empty response
 | 
											
												
													
														|  |              //(it happens with search across _all with no indices around and consistent with broadcast operations)
 |  |              //(it happens with search across _all with no indices around and consistent with broadcast operations)
 | 
											
												
													
														|  |              listener.onResponse(new SearchResponse(InternalSearchResponse.empty(), null, 0, 0, buildTookInMillis(),
 |  |              listener.onResponse(new SearchResponse(InternalSearchResponse.empty(), null, 0, 0, buildTookInMillis(),
 | 
											
												
													
														|  |                  ShardSearchFailure.EMPTY_ARRAY));
 |  |                  ShardSearchFailure.EMPTY_ARRAY));
 | 
											
												
													
														|  |              return;
 |  |              return;
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  | -        int shardIndex = -1;
 |  | 
 | 
											
												
													
														|  | -        for (final ShardIterator shardIt : shardsIts) {
 |  | 
 | 
											
												
													
														|  | -            shardIndex++;
 |  | 
 | 
											
												
													
														|  | -            final ShardRouting shard = shardIt.nextOrNull();
 |  | 
 | 
											
												
													
														|  | -            if (shard != null) {
 |  | 
 | 
											
												
													
														|  | -                performInitialPhase(shardIndex, shardIt, shard);
 |  | 
 | 
											
												
													
														|  | -            } else {
 |  | 
 | 
											
												
													
														|  | -                // really, no shards active in this group
 |  | 
 | 
											
												
													
														|  | -                onInitialPhaseResult(shardIndex, null, null, shardIt, new NoShardAvailableActionException(shardIt.shardId()));
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +        executePhase(this);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    void performInitialPhase(final int shardIndex, final ShardIterator shardIt, final ShardRouting shard) {
 |  | 
 | 
											
												
													
														|  | -        if (shard == null) {
 |  | 
 | 
											
												
													
														|  | -            // TODO upgrade this to an assert...
 |  | 
 | 
											
												
													
														|  | -            // no more active shards... (we should not really get here, but just for safety)
 |  | 
 | 
											
												
													
														|  | -            onInitialPhaseResult(shardIndex, null, null, shardIt, new NoShardAvailableActionException(shardIt.shardId()));
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPhase) {
 | 
											
												
													
														|  | 
 |  | +        /* This is the main search phase transition where we move to the next phase. At this point we check if there is
 | 
											
												
													
														|  | 
 |  | +         * at least one successful operation left and if so we move to the next phase. If not we immediately fail the
 | 
											
												
													
														|  | 
 |  | +         * search phase as "all shards failed"*/
 | 
											
												
													
														|  | 
 |  | +        if (successfulOps.get() == 0) { // we have 0 successful results that means we shortcut stuff and return a failure
 | 
											
												
													
														|  | 
 |  | +            if (logger.isDebugEnabled()) {
 | 
											
												
													
														|  | 
 |  | +                final ShardOperationFailedException[] shardSearchFailures = ExceptionsHelper.groupBy(buildShardFailures());
 | 
											
												
													
														|  | 
 |  | +                Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0];
 | 
											
												
													
														|  | 
 |  | +                logger.debug((Supplier<?>) () -> new ParameterizedMessage("All shards failed for phase: [{}]", getName()),
 | 
											
												
													
														|  | 
 |  | +                    cause);
 | 
											
												
													
														|  | 
 |  | +            }
 | 
											
												
													
														|  | 
 |  | +            onPhaseFailure(currentPhase, "all shards failed", null);
 | 
											
												
													
														|  |          } else {
 |  |          } else {
 | 
											
												
													
														|  | -            try {
 |  | 
 | 
											
												
													
														|  | -                final Transport.Connection connection = nodeIdToConnection.apply(shard.currentNodeId());
 |  | 
 | 
											
												
													
														|  | -                AliasFilter filter = this.aliasFilter.get(shard.index().getUUID());
 |  | 
 | 
											
												
													
														|  | -                assert filter != null;
 |  | 
 | 
											
												
													
														|  | -                float indexBoost = concreteIndexBoosts.getOrDefault(shard.index().getUUID(), DEFAULT_INDEX_BOOST);
 |  | 
 | 
											
												
													
														|  | -                ShardSearchTransportRequest transportRequest = new ShardSearchTransportRequest(request, shardIt.shardId(), shardsIts.size(),
 |  | 
 | 
											
												
													
														|  | -                    filter, indexBoost, startTime());
 |  | 
 | 
											
												
													
														|  | -                sendExecuteFirstPhase(connection, transportRequest, new ActionListener<FirstResult>() {
 |  | 
 | 
											
												
													
														|  | -                    @Override
 |  | 
 | 
											
												
													
														|  | -                    public void onResponse(FirstResult result) {
 |  | 
 | 
											
												
													
														|  | -                        onInitialPhaseResult(shardIndex, shard.currentNodeId(), result, shardIt);
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -                    @Override
 |  | 
 | 
											
												
													
														|  | -                    public void onFailure(Exception t) {
 |  | 
 | 
											
												
													
														|  | -                        onInitialPhaseResult(shardIndex, shard, connection.getNode().getId(), shardIt, t);
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -                });
 |  | 
 | 
											
												
													
														|  | -            } catch (ConnectTransportException | IllegalArgumentException ex) {
 |  | 
 | 
											
												
													
														|  | -                // we are getting the connection early here so we might run into nodes that are not connected. in that case we move on to
 |  | 
 | 
											
												
													
														|  | -                // the next shard. previously when using discovery nodes here we had a special case for null when a node was not connected
 |  | 
 | 
											
												
													
														|  | -                // at all which is not not needed anymore.
 |  | 
 | 
											
												
													
														|  | -                onInitialPhaseResult(shardIndex, shard, shard.currentNodeId(), shardIt, ex);
 |  | 
 | 
											
												
													
														|  | 
 |  | +            if (logger.isTraceEnabled()) {
 | 
											
												
													
														|  | 
 |  | +                final String resultsFrom = results.asList().stream()
 | 
											
												
													
														|  | 
 |  | +                    .map(r -> r.value.shardTarget().toString()).collect(Collectors.joining(","));
 | 
											
												
													
														|  | 
 |  | +                logger.trace("[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})",
 | 
											
												
													
														|  | 
 |  | +                    currentPhase.getName(), nextPhase.getName(), resultsFrom, clusterStateVersion);
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  | 
 |  | +            executePhase(nextPhase);
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    private void onInitialPhaseResult(int shardIndex, String nodeId, FirstResult result, ShardIterator shardIt) {
 |  | 
 | 
											
												
													
														|  | -        result.shardTarget(new SearchShardTarget(nodeId, shardIt.shardId()));
 |  | 
 | 
											
												
													
														|  | -        processFirstPhaseResult(shardIndex, result);
 |  | 
 | 
											
												
													
														|  | -        // we need to increment successful ops first before we compare the exit condition otherwise if we
 |  | 
 | 
											
												
													
														|  | -        // are fast we could concurrently update totalOps but then preempt one of the threads which can
 |  | 
 | 
											
												
													
														|  | -        // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc.
 |  | 
 | 
											
												
													
														|  | -        successfulOps.incrementAndGet();
 |  | 
 | 
											
												
													
														|  | -        // increment all the "future" shards to update the total ops since we some may work and some may not...
 |  | 
 | 
											
												
													
														|  | -        // and when that happens, we break on total ops, so we must maintain them
 |  | 
 | 
											
												
													
														|  | -        final int xTotalOps = totalOps.addAndGet(shardIt.remaining() + 1);
 |  | 
 | 
											
												
													
														|  | -        if (xTotalOps == expectedTotalOps) {
 |  | 
 | 
											
												
													
														|  | -            executePhase(initialPhaseName(), innerGetNextPhase(), null);
 |  | 
 | 
											
												
													
														|  | -        } else if (xTotalOps > expectedTotalOps) {
 |  | 
 | 
											
												
													
														|  | -            // this is fatal - something is completely wrong here?
 |  | 
 | 
											
												
													
														|  | -            throw new AssertionError( "unexpected higher total ops [" + xTotalOps + "] compared to expected ["
 |  | 
 | 
											
												
													
														|  | -                + expectedTotalOps + "]");
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected void executePhase(String phaseName, CheckedRunnable<Exception> phase, Exception suppressedException) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private void executePhase(SearchPhase phase) {
 | 
											
												
													
														|  |          try {
 |  |          try {
 | 
											
												
													
														|  |              phase.run();
 |  |              phase.run();
 | 
											
												
													
														|  |          } catch (Exception e) {
 |  |          } catch (Exception e) {
 | 
											
												
													
														|  | -            if (suppressedException != null) {
 |  | 
 | 
											
												
													
														|  | -                e.addSuppressed(suppressedException);
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  |              if (logger.isDebugEnabled()) {
 |  |              if (logger.isDebugEnabled()) {
 | 
											
												
													
														|  |                  logger.debug(
 |  |                  logger.debug(
 | 
											
												
													
														|  |                      (Supplier<?>) () -> new ParameterizedMessage(
 |  |                      (Supplier<?>) () -> new ParameterizedMessage(
 | 
											
												
													
														|  | -                        "Failed to execute [{}] while moving to second phase", request),
 |  | 
 | 
											
												
													
														|  | 
 |  | +                        "Failed to execute [{}] while moving to [{}] phase", request, phase.getName()),
 | 
											
												
													
														|  |                      e);
 |  |                      e);
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  | -            raisePhaseFailure(new ReduceSearchPhaseException(phaseName, "", e, buildShardFailures()));
 |  | 
 | 
											
												
													
														|  | 
 |  | +            onPhaseFailure(phase, "", e);
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    private void onInitialPhaseResult(final int shardIndex, @Nullable ShardRouting shard, @Nullable String nodeId,
 |  | 
 | 
											
												
													
														|  | -                                      final ShardIterator shardIt, Exception e) {
 |  | 
 | 
											
												
													
														|  | -        // we always add the shard failure for a specific shard instance
 |  | 
 | 
											
												
													
														|  | -        // we do make sure to clean it on a successful response from a shard
 |  | 
 | 
											
												
													
														|  | -        SearchShardTarget shardTarget = new SearchShardTarget(nodeId, shardIt.shardId());
 |  | 
 | 
											
												
													
														|  | -        addShardFailure(shardIndex, shardTarget, e);
 |  | 
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        if (totalOps.incrementAndGet() == expectedTotalOps) {
 |  | 
 | 
											
												
													
														|  | -            if (logger.isDebugEnabled()) {
 |  | 
 | 
											
												
													
														|  | -                if (e != null && !TransportActions.isShardNotAvailableException(e)) {
 |  | 
 | 
											
												
													
														|  | -                    logger.debug(
 |  | 
 | 
											
												
													
														|  | -                        (Supplier<?>) () -> new ParameterizedMessage(
 |  | 
 | 
											
												
													
														|  | -                            "{}: Failed to execute [{}]",
 |  | 
 | 
											
												
													
														|  | -                            shard != null ? shard.shortSummary() :
 |  | 
 | 
											
												
													
														|  | -                                shardIt.shardId(),
 |  | 
 | 
											
												
													
														|  | -                            request),
 |  | 
 | 
											
												
													
														|  | -                        e);
 |  | 
 | 
											
												
													
														|  | -                } else if (logger.isTraceEnabled()) {
 |  | 
 | 
											
												
													
														|  | -                    logger.trace((Supplier<?>) () -> new ParameterizedMessage("{}: Failed to execute [{}]", shard, request), e);
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -            final ShardSearchFailure[] shardSearchFailures = buildShardFailures();
 |  | 
 | 
											
												
													
														|  | -            if (successfulOps.get() == 0) {
 |  | 
 | 
											
												
													
														|  | -                if (logger.isDebugEnabled()) {
 |  | 
 | 
											
												
													
														|  | -                    logger.debug((Supplier<?>) () -> new ParameterizedMessage("All shards failed for phase: [{}]", initialPhaseName()), e);
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -                // no successful ops, raise an exception
 |  | 
 | 
											
												
													
														|  | -                raisePhaseFailure(new SearchPhaseExecutionException(initialPhaseName(), "all shards failed", e, shardSearchFailures));
 |  | 
 | 
											
												
													
														|  | -            } else {
 |  | 
 | 
											
												
													
														|  | -                executePhase(initialPhaseName(), innerGetNextPhase(), e);
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        } else {
 |  | 
 | 
											
												
													
														|  | -            final ShardRouting nextShard = shardIt.nextOrNull();
 |  | 
 | 
											
												
													
														|  | -            final boolean lastShard = nextShard == null;
 |  | 
 | 
											
												
													
														|  | -            // trace log this exception
 |  | 
 | 
											
												
													
														|  | -            logger.trace(
 |  | 
 | 
											
												
													
														|  | -                (Supplier<?>) () -> new ParameterizedMessage(
 |  | 
 | 
											
												
													
														|  | -                    "{}: Failed to execute [{}] lastShard [{}]",
 |  | 
 | 
											
												
													
														|  | -                    shard != null ? shard.shortSummary() : shardIt.shardId(),
 |  | 
 | 
											
												
													
														|  | -                    request,
 |  | 
 | 
											
												
													
														|  | -                    lastShard),
 |  | 
 | 
											
												
													
														|  | -                e);
 |  | 
 | 
											
												
													
														|  | -            if (!lastShard) {
 |  | 
 | 
											
												
													
														|  | -                try {
 |  | 
 | 
											
												
													
														|  | -                    performInitialPhase(shardIndex, shardIt, nextShard);
 |  | 
 | 
											
												
													
														|  | -                } catch (Exception inner) {
 |  | 
 | 
											
												
													
														|  | -                    inner.addSuppressed(e);
 |  | 
 | 
											
												
													
														|  | -                    onInitialPhaseResult(shardIndex, shard, shard.currentNodeId(), shardIt, inner);
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            } else {
 |  | 
 | 
											
												
													
														|  | -                // no more shards active, add a failure
 |  | 
 | 
											
												
													
														|  | -                if (logger.isDebugEnabled() && !logger.isTraceEnabled()) { // do not double log this exception
 |  | 
 | 
											
												
													
														|  | -                    if (e != null && !TransportActions.isShardNotAvailableException(e)) {
 |  | 
 | 
											
												
													
														|  | -                        logger.debug(
 |  | 
 | 
											
												
													
														|  | -                            (Supplier<?>) () -> new ParameterizedMessage(
 |  | 
 | 
											
												
													
														|  | -                                "{}: Failed to execute [{}] lastShard [{}]",
 |  | 
 | 
											
												
													
														|  | -                                shard != null ? shard.shortSummary() :
 |  | 
 | 
											
												
													
														|  | -                                    shardIt.shardId(),
 |  | 
 | 
											
												
													
														|  | -                                request,
 |  | 
 | 
											
												
													
														|  | -                                lastShard),
 |  | 
 | 
											
												
													
														|  | -                            e);
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected final ShardSearchFailure[] buildShardFailures() {
 |  | 
 | 
											
												
													
														|  | -        AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private ShardSearchFailure[] buildShardFailures() {
 | 
											
												
													
														|  | 
 |  | +        AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures.get();
 | 
											
												
													
														|  |          if (shardFailures == null) {
 |  |          if (shardFailures == null) {
 | 
											
												
													
														|  |              return ShardSearchFailure.EMPTY_ARRAY;
 |  |              return ShardSearchFailure.EMPTY_ARRAY;
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
										
											
												
													
														|  | @@ -274,17 +167,19 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
 | 
											
												
													
														|  |          return failures;
 |  |          return failures;
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    protected final void addShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +    public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) {
 | 
											
												
													
														|  |          // we don't aggregate shard failures on non active shards (but do keep the header counts right)
 |  |          // we don't aggregate shard failures on non active shards (but do keep the header counts right)
 | 
											
												
													
														|  |          if (TransportActions.isShardNotAvailableException(e)) {
 |  |          if (TransportActions.isShardNotAvailableException(e)) {
 | 
											
												
													
														|  |              return;
 |  |              return;
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | 
 |  | +        AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures.get();
 | 
											
												
													
														|  |          // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures)
 |  |          // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures)
 | 
											
												
													
														|  | -        if (shardFailures == null) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +        if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally
 | 
											
												
													
														|  |              synchronized (shardFailuresMutex) {
 |  |              synchronized (shardFailuresMutex) {
 | 
											
												
													
														|  | -                if (shardFailures == null) {
 |  | 
 | 
											
												
													
														|  | -                    shardFailures = new AtomicArray<>(shardsIts.size());
 |  | 
 | 
											
												
													
														|  | 
 |  | +                shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it?
 | 
											
												
													
														|  | 
 |  | +                if (shardFailures == null) { // still null so we are the first and create a new instance
 | 
											
												
													
														|  | 
 |  | +                    shardFailures = new AtomicArray<>(results.length());
 | 
											
												
													
														|  | 
 |  | +                    this.shardFailures.set(shardFailures);
 | 
											
												
													
														|  |                  }
 |  |                  }
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
										
											
												
													
														|  | @@ -298,15 +193,21 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
 | 
											
												
													
														|  |                  shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget));
 |  |                  shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget));
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +        if (results.get(shardIndex) != null) {
 | 
											
												
													
														|  | 
 |  | +            assert failure == null : "shard failed before but shouldn't: " + failure;
 | 
											
												
													
														|  | 
 |  | +            successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter
 | 
											
												
													
														|  | 
 |  | +        }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |      /**
 |  |      /**
 | 
											
												
													
														|  |       * This method should be called if a search phase failed to ensure all relevant search contexts and resources are released.
 |  |       * This method should be called if a search phase failed to ensure all relevant search contexts and resources are released.
 | 
											
												
													
														|  |       * this method will also notify the listener and sends back a failure to the user.
 |  |       * this method will also notify the listener and sends back a failure to the user.
 | 
											
												
													
														|  | 
 |  | +     *
 | 
											
												
													
														|  |       * @param exception the exception explaining or causing the phase failure
 |  |       * @param exception the exception explaining or causing the phase failure
 | 
											
												
													
														|  |       */
 |  |       */
 | 
											
												
													
														|  | -    protected void raisePhaseFailure(SearchPhaseExecutionException exception) {
 |  | 
 | 
											
												
													
														|  | -        for (AtomicArray.Entry<FirstResult> entry : initialResults.asList()) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private void raisePhaseFailure(SearchPhaseExecutionException exception) {
 | 
											
												
													
														|  | 
 |  | +        for (AtomicArray.Entry<Result> entry : results.asList()) {
 | 
											
												
													
														|  |              try {
 |  |              try {
 | 
											
												
													
														|  |                  Transport.Connection connection = nodeIdToConnection.apply(entry.value.shardTarget().getNodeId());
 |  |                  Transport.Connection connection = nodeIdToConnection.apply(entry.value.shardTarget().getNodeId());
 | 
											
												
													
														|  |                  sendReleaseSearchContext(entry.value.id(), connection);
 |  |                  sendReleaseSearchContext(entry.value.id(), connection);
 | 
											
										
											
												
													
														|  | @@ -318,254 +219,97 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
 | 
											
												
													
														|  |          listener.onFailure(exception);
 |  |          listener.onFailure(exception);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    protected void sendReleaseSearchContext(long contextId, Transport.Connection connection) {
 |  | 
 | 
											
												
													
														|  | -        if (connection != null) {
 |  | 
 | 
											
												
													
														|  | -            searchTransportService.sendFreeContext(connection, contextId, request);
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected ShardFetchSearchRequest createFetchRequest(long queryId, int index, IntArrayList entry,
 |  | 
 | 
											
												
													
														|  | -                                                         ScoreDoc[] lastEmittedDocPerShard) {
 |  | 
 | 
											
												
													
														|  | -        final ScoreDoc lastEmittedDoc = (lastEmittedDocPerShard != null) ? lastEmittedDocPerShard[index] : null;
 |  | 
 | 
											
												
													
														|  | -        return new ShardFetchSearchRequest(request, queryId, entry, lastEmittedDoc);
 |  | 
 | 
											
												
													
														|  | -    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected abstract void sendExecuteFirstPhase(Transport.Connection connection, ShardSearchTransportRequest request,
 |  | 
 | 
											
												
													
														|  | -                                                  ActionListener<FirstResult> listener);
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected final void processFirstPhaseResult(int shardIndex, FirstResult result) {
 |  | 
 | 
											
												
													
														|  | -        initialResults.set(shardIndex, result);
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void onShardSuccess(int shardIndex, Result result) {
 | 
											
												
													
														|  | 
 |  | +        successfulOps.incrementAndGet();
 | 
											
												
													
														|  | 
 |  | +        results.set(shardIndex, result);
 | 
											
												
													
														|  |          if (logger.isTraceEnabled()) {
 |  |          if (logger.isTraceEnabled()) {
 | 
											
												
													
														|  |              logger.trace("got first-phase result from {}", result != null ? result.shardTarget() : null);
 |  |              logger.trace("got first-phase result from {}", result != null ? result.shardTarget() : null);
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |          // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level
 |  |          // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level
 | 
											
												
													
														|  |          // so its ok concurrency wise to miss potentially the shard failures being created because of another failure
 |  |          // so its ok concurrency wise to miss potentially the shard failures being created because of another failure
 | 
											
												
													
														|  |          // in the #addShardFailure, because by definition, it will happen on *another* shardIndex
 |  |          // in the #addShardFailure, because by definition, it will happen on *another* shardIndex
 | 
											
												
													
														|  | -        AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures;
 |  | 
 | 
											
												
													
														|  | 
 |  | +        AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures.get();
 | 
											
												
													
														|  |          if (shardFailures != null) {
 |  |          if (shardFailures != null) {
 | 
											
												
													
														|  |              shardFailures.set(shardIndex, null);
 |  |              shardFailures.set(shardIndex, null);
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    final CheckedRunnable<Exception> innerGetNextPhase() {
 |  | 
 | 
											
												
													
														|  | -        if (logger.isTraceEnabled()) {
 |  | 
 | 
											
												
													
														|  | -            StringBuilder sb = new StringBuilder();
 |  | 
 | 
											
												
													
														|  | -            boolean hadOne = false;
 |  | 
 | 
											
												
													
														|  | -            for (int i = 0; i < initialResults.length(); i++) {
 |  | 
 | 
											
												
													
														|  | -                FirstResult result = initialResults.get(i);
 |  | 
 | 
											
												
													
														|  | -                if (result == null) {
 |  | 
 | 
											
												
													
														|  | -                    continue; // failure
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -                if (hadOne) {
 |  | 
 | 
											
												
													
														|  | -                    sb.append(",");
 |  | 
 | 
											
												
													
														|  | -                } else {
 |  | 
 | 
											
												
													
														|  | -                    hadOne = true;
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -                sb.append(result.shardTarget());
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -            logger.trace("Moving to second phase, based on results from: {} (cluster state version: {})", sb, clusterStateVersion);
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -        return getNextPhase(initialResults);
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void onPhaseDone() {
 | 
											
												
													
														|  | 
 |  | +        executeNextPhase(this, getNextPhase(results, this));
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    protected abstract CheckedRunnable<Exception> getNextPhase(AtomicArray<FirstResult> initialResults);
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected abstract String initialPhaseName();
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    protected Executor getExecutor() {
 |  | 
 | 
											
												
													
														|  | -        return executor;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final int getNumShards() {
 | 
											
												
													
														|  | 
 |  | +        return results.length();
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    // this is a simple base class to simplify fan out to shards and collect
 |  | 
 | 
											
												
													
														|  | -    final class CountedCollector<R extends SearchPhaseResult> {
 |  | 
 | 
											
												
													
														|  | -        private final AtomicArray<R> resultArray;
 |  | 
 | 
											
												
													
														|  | -        private final CountDown counter;
 |  | 
 | 
											
												
													
														|  | -        private final IntConsumer onFinish;
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -        CountedCollector(AtomicArray<R> resultArray, int expectedOps, IntConsumer onFinish) {
 |  | 
 | 
											
												
													
														|  | -            this.resultArray = resultArray;
 |  | 
 | 
											
												
													
														|  | -            this.counter = new CountDown(expectedOps);
 |  | 
 | 
											
												
													
														|  | -            this.onFinish = onFinish;
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -        void countDown() {
 |  | 
 | 
											
												
													
														|  | -            if (counter.countDown()) {
 |  | 
 | 
											
												
													
														|  | -                onFinish.accept(successfulOps.get());
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -        void onResult(int index, R result, SearchShardTarget target) {
 |  | 
 | 
											
												
													
														|  | -            try {
 |  | 
 | 
											
												
													
														|  | -                result.shardTarget(target);
 |  | 
 | 
											
												
													
														|  | -                resultArray.set(index, result);
 |  | 
 | 
											
												
													
														|  | -            } finally {
 |  | 
 | 
											
												
													
														|  | -                countDown();
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -        void onFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) {
 |  | 
 | 
											
												
													
														|  | -            try {
 |  | 
 | 
											
												
													
														|  | -                addShardFailure(shardIndex, shardTarget, e);
 |  | 
 | 
											
												
													
														|  | -            } finally {
 |  | 
 | 
											
												
													
														|  | -                successfulOps.decrementAndGet();
 |  | 
 | 
											
												
													
														|  | -                countDown();
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final Logger getLogger() {
 | 
											
												
													
														|  | 
 |  | +        return logger;
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final SearchTask getTask() {
 | 
											
												
													
														|  | 
 |  | +        return task;
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    /*
 |  | 
 | 
											
												
													
														|  | -     *  At this point AbstractSearchAsyncAction is just a base-class for the first phase of a search where we have multiple replicas
 |  | 
 | 
											
												
													
														|  | -     *  for each shardID. If one of them is not available we move to the next one. Yet, once we passed that first stage we have to work with
 |  | 
 | 
											
												
													
														|  | -     *  the shards we succeeded on the initial phase.
 |  | 
 | 
											
												
													
														|  | -     *  Unfortunately, subsequent phases are not fully detached from the initial phase since they are all non-static inner classes.
 |  | 
 | 
											
												
													
														|  | -     *  In future changes this will be changed to detach the inner classes to test them in isolation and to simplify their creation.
 |  | 
 | 
											
												
													
														|  | -     *  The AbstractSearchAsyncAction should be final and it should just get a factory for the next phase instead of requiring subclasses
 |  | 
 | 
											
												
													
														|  | -     *  etc.
 |  | 
 | 
											
												
													
														|  | -     */
 |  | 
 | 
											
												
													
														|  | -    final class FetchPhase implements CheckedRunnable<Exception> {
 |  | 
 | 
											
												
													
														|  | -        private final AtomicArray<FetchSearchResult> fetchResults;
 |  | 
 | 
											
												
													
														|  | -        private final SearchPhaseController searchPhaseController;
 |  | 
 | 
											
												
													
														|  | -        private final AtomicArray<QuerySearchResultProvider> queryResults;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final SearchRequest getRequest() {
 | 
											
												
													
														|  | 
 |  | +        return request;
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        FetchPhase(AtomicArray<QuerySearchResultProvider> queryResults,
 |  | 
 | 
											
												
													
														|  | -                           SearchPhaseController searchPhaseController) {
 |  | 
 | 
											
												
													
														|  | -            this.fetchResults = new AtomicArray<>(queryResults.length());
 |  | 
 | 
											
												
													
														|  | -            this.searchPhaseController = searchPhaseController;
 |  | 
 | 
											
												
													
														|  | -            this.queryResults = queryResults;
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
 | 
											
												
													
														|  | 
 |  | +        return new SearchResponse(internalSearchResponse, scrollId, results.length(), successfulOps.get(),
 | 
											
												
													
														|  | 
 |  | +            buildTookInMillis(), buildShardFailures());
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        @Override
 |  | 
 | 
											
												
													
														|  | -        public void run() throws Exception {
 |  | 
 | 
											
												
													
														|  | -            getExecutor().execute(new ActionRunnable<SearchResponse>(listener) {
 |  | 
 | 
											
												
													
														|  | -                @Override
 |  | 
 | 
											
												
													
														|  | -                public void doRun() throws IOException {
 |  | 
 | 
											
												
													
														|  | -                    // we do the heavy lifting in this inner run method where we reduce aggs etc. that's why we fork this phase
 |  | 
 | 
											
												
													
														|  | -                    // off immediately instead of forking when we send back the response to the user since there we only need
 |  | 
 | 
											
												
													
														|  | -                    // to merge together the fetched results which is a linear operation.
 |  | 
 | 
											
												
													
														|  | -                    innerRun();
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) {
 | 
											
												
													
														|  | 
 |  | +        raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures()));
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -                @Override
 |  | 
 | 
											
												
													
														|  | -                public void onFailure(Exception e) {
 |  | 
 | 
											
												
													
														|  | -                    raisePhaseFailure(new ReduceSearchPhaseException("fetch", "", e, buildShardFailures()));
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            });
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final Transport.Connection getConnection(String nodeId) {
 | 
											
												
													
														|  | 
 |  | +        return nodeIdToConnection.apply(nodeId);
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        private void innerRun() throws IOException{
 |  | 
 | 
											
												
													
														|  | -            final int numShards = shardsIts.size();
 |  | 
 | 
											
												
													
														|  | -            final boolean isScrollRequest = request.scroll() != null;
 |  | 
 | 
											
												
													
														|  | -            ScoreDoc[] sortedShardDocs = searchPhaseController.sortDocs(isScrollRequest, queryResults);
 |  | 
 | 
											
												
													
														|  | -            String scrollId = isScrollRequest ? TransportSearchHelper.buildScrollId(queryResults) : null;
 |  | 
 | 
											
												
													
														|  | -            List<AtomicArray.Entry<QuerySearchResultProvider>> queryResultsAsList = queryResults.asList();
 |  | 
 | 
											
												
													
														|  | -            final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = searchPhaseController.reducedQueryPhase(queryResultsAsList);
 |  | 
 | 
											
												
													
														|  | -            final boolean queryAndFetchOptimization = queryResults.length() == 1;
 |  | 
 | 
											
												
													
														|  | -            final IntConsumer finishPhase = successOpts
 |  | 
 | 
											
												
													
														|  | -                -> sendResponse(searchPhaseController, sortedShardDocs, scrollId, reducedQueryPhase, queryAndFetchOptimization ?
 |  | 
 | 
											
												
													
														|  | -                    queryResults : fetchResults);
 |  | 
 | 
											
												
													
														|  | -            if (queryAndFetchOptimization) {
 |  | 
 | 
											
												
													
														|  | -                assert queryResults.get(0) == null || queryResults.get(0).fetchResult() != null;
 |  | 
 | 
											
												
													
														|  | -                // query AND fetch optimization
 |  | 
 | 
											
												
													
														|  | -                finishPhase.accept(successfulOps.get());
 |  | 
 | 
											
												
													
														|  | -            } else {
 |  | 
 | 
											
												
													
														|  | -                final IntArrayList[] docIdsToLoad = searchPhaseController.fillDocIdsToLoad(numShards, sortedShardDocs);
 |  | 
 | 
											
												
													
														|  | -                if (sortedShardDocs.length == 0) { // no docs to fetch -- sidestep everything and return
 |  | 
 | 
											
												
													
														|  | -                    queryResultsAsList.stream()
 |  | 
 | 
											
												
													
														|  | -                        .map(e -> e.value.queryResult())
 |  | 
 | 
											
												
													
														|  | -                        .forEach(this::releaseIrrelevantSearchContext); // we have to release contexts here to free up resources
 |  | 
 | 
											
												
													
														|  | -                    finishPhase.accept(successfulOps.get());
 |  | 
 | 
											
												
													
														|  | -                } else {
 |  | 
 | 
											
												
													
														|  | -                    final ScoreDoc[] lastEmittedDocPerShard = isScrollRequest ?
 |  | 
 | 
											
												
													
														|  | -                        searchPhaseController.getLastEmittedDocPerShard(reducedQueryPhase, sortedShardDocs, numShards)
 |  | 
 | 
											
												
													
														|  | -                        : null;
 |  | 
 | 
											
												
													
														|  | -                    final CountedCollector<FetchSearchResult> counter = new CountedCollector<>(fetchResults,
 |  | 
 | 
											
												
													
														|  | -                        docIdsToLoad.length, // we count down every shard in the result no matter if we got any results or not
 |  | 
 | 
											
												
													
														|  | -                        finishPhase);
 |  | 
 | 
											
												
													
														|  | -                    for (int i = 0; i < docIdsToLoad.length; i++) {
 |  | 
 | 
											
												
													
														|  | -                        IntArrayList entry = docIdsToLoad[i];
 |  | 
 | 
											
												
													
														|  | -                        QuerySearchResultProvider queryResult = queryResults.get(i);
 |  | 
 | 
											
												
													
														|  | -                        if (entry == null) { // no results for this shard ID
 |  | 
 | 
											
												
													
														|  | -                            if (queryResult != null) {
 |  | 
 | 
											
												
													
														|  | -                                // if we got some hits from this shard we have to release the context there
 |  | 
 | 
											
												
													
														|  | -                                // we do this as we go since it will free up resources and passing on the request on the
 |  | 
 | 
											
												
													
														|  | -                                // transport layer is cheap.
 |  | 
 | 
											
												
													
														|  | -                                releaseIrrelevantSearchContext(queryResult.queryResult());
 |  | 
 | 
											
												
													
														|  | -                            }
 |  | 
 | 
											
												
													
														|  | -                            // in any case we count down this result since we don't talk to this shard anymore
 |  | 
 | 
											
												
													
														|  | -                            counter.countDown();
 |  | 
 | 
											
												
													
														|  | -                        } else {
 |  | 
 | 
											
												
													
														|  | -                            Transport.Connection connection = nodeIdToConnection.apply(queryResult.shardTarget().getNodeId());
 |  | 
 | 
											
												
													
														|  | -                            ShardFetchSearchRequest fetchSearchRequest = createFetchRequest(queryResult.queryResult().id(), i, entry,
 |  | 
 | 
											
												
													
														|  | -                                lastEmittedDocPerShard);
 |  | 
 | 
											
												
													
														|  | -                            executeFetch(i, queryResult.shardTarget(), counter, fetchSearchRequest, queryResult.queryResult(),
 |  | 
 | 
											
												
													
														|  | -                                connection);
 |  | 
 | 
											
												
													
														|  | -                        }
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final SearchTransportService getSearchTransport() {
 | 
											
												
													
														|  | 
 |  | +        return searchTransportService;
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        private void executeFetch(final int shardIndex, final SearchShardTarget shardTarget,
 |  | 
 | 
											
												
													
														|  | -                                    final CountedCollector<FetchSearchResult> counter,
 |  | 
 | 
											
												
													
														|  | -                                    final ShardFetchSearchRequest fetchSearchRequest, final QuerySearchResult querySearchResult,
 |  | 
 | 
											
												
													
														|  | -                                    final Transport.Connection connection) {
 |  | 
 | 
											
												
													
														|  | -            searchTransportService.sendExecuteFetch(connection, fetchSearchRequest, task, new ActionListener<FetchSearchResult>() {
 |  | 
 | 
											
												
													
														|  | -                @Override
 |  | 
 | 
											
												
													
														|  | -                public void onResponse(FetchSearchResult result) {
 |  | 
 | 
											
												
													
														|  | -                    counter.onResult(shardIndex, result, shardTarget);
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void execute(Runnable command) {
 | 
											
												
													
														|  | 
 |  | +        executor.execute(command);
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -                @Override
 |  | 
 | 
											
												
													
														|  | -                public void onFailure(Exception e) {
 |  | 
 | 
											
												
													
														|  | -                    try {
 |  | 
 | 
											
												
													
														|  | -                        if (logger.isDebugEnabled()) {
 |  | 
 | 
											
												
													
														|  | -                            logger.debug((Supplier<?>) () -> new ParameterizedMessage("[{}] Failed to execute fetch phase",
 |  | 
 | 
											
												
													
														|  | -                                fetchSearchRequest.id()), e);
 |  | 
 | 
											
												
													
														|  | -                        }
 |  | 
 | 
											
												
													
														|  | -                        counter.onFailure(shardIndex, shardTarget, e);
 |  | 
 | 
											
												
													
														|  | -                    } finally {
 |  | 
 | 
											
												
													
														|  | -                        // the search context might not be cleared on the node where the fetch was executed for example
 |  | 
 | 
											
												
													
														|  | -                        // because the action was rejected by the thread pool. in this case we need to send a dedicated
 |  | 
 | 
											
												
													
														|  | -                        // request to clear the search context.
 |  | 
 | 
											
												
													
														|  | -                        releaseIrrelevantSearchContext(querySearchResult);
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            });
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void onResponse(SearchResponse response) {
 | 
											
												
													
														|  | 
 |  | +        listener.onResponse(response);
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        /**
 |  | 
 | 
											
												
													
														|  | -         * Releases shard targets that are not used in the docsIdsToLoad.
 |  | 
 | 
											
												
													
														|  | -         */
 |  | 
 | 
											
												
													
														|  | -        private void releaseIrrelevantSearchContext(QuerySearchResult queryResult) {
 |  | 
 | 
											
												
													
														|  | -            // we only release search context that we did not fetch from if we are not scrolling
 |  | 
 | 
											
												
													
														|  | -            // and if it has at lease one hit that didn't make it to the global topDocs
 |  | 
 | 
											
												
													
														|  | -            if (request.scroll() == null && queryResult.hasHits()) {
 |  | 
 | 
											
												
													
														|  | -                try {
 |  | 
 | 
											
												
													
														|  | -                    Transport.Connection connection = nodeIdToConnection.apply(queryResult.shardTarget().getNodeId());
 |  | 
 | 
											
												
													
														|  | -                    sendReleaseSearchContext(queryResult.id(), connection);
 |  | 
 | 
											
												
													
														|  | -                } catch (Exception e) {
 |  | 
 | 
											
												
													
														|  | -                    logger.trace("failed to release context", e);
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    @Override
 | 
											
												
													
														|  | 
 |  | +    public final void onFailure(Exception e) {
 | 
											
												
													
														|  | 
 |  | +        listener.onFailure(e);
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -        /**
 |  | 
 | 
											
												
													
														|  | -         * Sends back a result to the user.
 |  | 
 | 
											
												
													
														|  | -         */
 |  | 
 | 
											
												
													
														|  | -        private void sendResponse(SearchPhaseController searchPhaseController, ScoreDoc[] sortedDocs,
 |  | 
 | 
											
												
													
														|  | -                                String scrollId, SearchPhaseController.ReducedQueryPhase reducedQueryPhase,
 |  | 
 | 
											
												
													
														|  | -                                AtomicArray<? extends QuerySearchResultProvider> fetchResultsArr) {
 |  | 
 | 
											
												
													
														|  | -            final boolean isScrollRequest = request.scroll() != null;
 |  | 
 | 
											
												
													
														|  | -            final InternalSearchResponse internalResponse = searchPhaseController.merge(isScrollRequest, sortedDocs, reducedQueryPhase,
 |  | 
 | 
											
												
													
														|  | -                fetchResultsArr);
 |  | 
 | 
											
												
													
														|  | -            listener.onResponse(new SearchResponse(internalResponse, scrollId, expectedSuccessfulOps, successfulOps.get(),
 |  | 
 | 
											
												
													
														|  | -                buildTookInMillis(), buildShardFailures()));
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | 
 |  | +    public final ShardSearchTransportRequest buildShardSearchRequest(ShardIterator shardIt, ShardRouting shard) {
 | 
											
												
													
														|  | 
 |  | +        AliasFilter filter = aliasFilter.get(shard.index().getUUID());
 | 
											
												
													
														|  | 
 |  | +        assert filter != null;
 | 
											
												
													
														|  | 
 |  | +        float indexBoost = concreteIndexBoosts.getOrDefault(shard.index().getUUID(), DEFAULT_INDEX_BOOST);
 | 
											
												
													
														|  | 
 |  | +        return new ShardSearchTransportRequest(request, shardIt.shardId(), getNumShards(),
 | 
											
												
													
														|  | 
 |  | +            filter, indexBoost, startTime);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | 
 |  | +    /**
 | 
											
												
													
														|  | 
 |  | +     * Returns the next phase based on the results of the initial search phase
 | 
											
												
													
														|  | 
 |  | +     * @param results the results of the initial search phase. Each non null element in the result array represent a successfully
 | 
											
												
													
														|  | 
 |  | +     *                executed shard request
 | 
											
												
													
														|  | 
 |  | +     * @param context the search context for the next phase
 | 
											
												
													
														|  | 
 |  | +     */
 | 
											
												
													
														|  | 
 |  | +    protected abstract SearchPhase getNextPhase(AtomicArray<Result> results, SearchPhaseContext context);
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  }
 |  |  }
 |