summaryrefslogtreecommitdiff
path: root/core/src/main/java/org/elasticsearch/action/percolate/TransportMultiPercolateAction.java
blob: 987ca3c4cd9964de19c955232734174e3d5dc2a9 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
/*
 * Licensed to Elasticsearch under one or more contributor
 * license agreements. See the NOTICE file distributed with
 * this work for additional information regarding copyright
 * ownership. Elasticsearch licenses this file to you under
 * the Apache License, Version 2.0 (the "License"); you may
 * not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing,
 * software distributed under the License is distributed on an
 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 * KIND, either express or implied.  See the License for the
 * specific language governing permissions and limitations
 * under the License.
 */

package org.elasticsearch.action.percolate;

import com.carrotsearch.hppc.IntArrayList;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.get.MultiGetItemResponse;
import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.action.get.TransportMultiGetAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.engine.DocumentMissingException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReferenceArray;

/**
 */
public class TransportMultiPercolateAction extends HandledTransportAction<MultiPercolateRequest, MultiPercolateResponse> {

    private final ClusterService clusterService;
    private final PercolatorService percolatorService;

    private final TransportMultiGetAction multiGetAction;
    private final TransportShardMultiPercolateAction shardMultiPercolateAction;

    @Inject
    public TransportMultiPercolateAction(Settings settings, ThreadPool threadPool, TransportShardMultiPercolateAction shardMultiPercolateAction,
                                         ClusterService clusterService, TransportService transportService, PercolatorService percolatorService,
                                         TransportMultiGetAction multiGetAction, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
        super(settings, MultiPercolateAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, MultiPercolateRequest::new);
        this.shardMultiPercolateAction = shardMultiPercolateAction;
        this.clusterService = clusterService;
        this.percolatorService = percolatorService;
        this.multiGetAction = multiGetAction;
    }

    @Override
    protected void doExecute(final MultiPercolateRequest request, final ActionListener<MultiPercolateResponse> listener) {
        final ClusterState clusterState = clusterService.state();
        clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ);

        final List<Object> percolateRequests = new ArrayList<>(request.requests().size());
        // Can have a mixture of percolate requests. (normal percolate requests & percolate existing doc),
        // so we need to keep track for what percolate request we had a get request
        final IntArrayList getRequestSlots = new IntArrayList();
        List<GetRequest> existingDocsRequests = new ArrayList<>();
        for (int slot = 0;  slot < request.requests().size(); slot++) {
            PercolateRequest percolateRequest = request.requests().get(slot);
            percolateRequest.startTime = System.currentTimeMillis();
            percolateRequests.add(percolateRequest);
            if (percolateRequest.getRequest() != null) {
                existingDocsRequests.add(percolateRequest.getRequest());
                getRequestSlots.add(slot);
            }
        }

        if (!existingDocsRequests.isEmpty()) {
            final MultiGetRequest multiGetRequest = new MultiGetRequest();
            for (GetRequest getRequest : existingDocsRequests) {
                multiGetRequest.add(
                        new MultiGetRequest.Item(getRequest.index(), getRequest.type(), getRequest.id())
                        .routing(getRequest.routing())
                );
            }

            multiGetAction.execute(multiGetRequest, new ActionListener<MultiGetResponse>() {

                @Override
                public void onResponse(MultiGetResponse multiGetItemResponses) {
                    for (int i = 0; i < multiGetItemResponses.getResponses().length; i++) {
                        MultiGetItemResponse itemResponse = multiGetItemResponses.getResponses()[i];
                        int slot = getRequestSlots.get(i);
                        if (!itemResponse.isFailed()) {
                            GetResponse getResponse = itemResponse.getResponse();
                            if (getResponse.isExists()) {
                                PercolateRequest originalRequest = (PercolateRequest) percolateRequests.get(slot);
                                percolateRequests.set(slot, new PercolateRequest(originalRequest, getResponse.getSourceAsBytesRef()));
                            } else {
                                logger.trace("mpercolate existing doc, item[{}] doesn't exist", slot);
                                percolateRequests.set(slot, new DocumentMissingException(null, getResponse.getType(), getResponse.getId()));
                            }
                        } else {
                            logger.trace("mpercolate existing doc, item[{}] failure {}", slot, itemResponse.getFailure());
                            percolateRequests.set(slot, itemResponse.getFailure());
                        }
                    }
                    new ASyncAction(request, percolateRequests, listener, clusterState).run();
                }

                @Override
                public void onFailure(Throwable e) {
                    listener.onFailure(e);
                }
            });
        } else {
            new ASyncAction(request, percolateRequests, listener, clusterState).run();
        }

    }

    private final class ASyncAction {

        final ActionListener<MultiPercolateResponse> finalListener;
        final Map<ShardId, TransportShardMultiPercolateAction.Request> requestsByShard;
        final MultiPercolateRequest multiPercolateRequest;
        final List<Object> percolateRequests;

        final Map<ShardId, IntArrayList> shardToSlots;
        final AtomicInteger expectedOperations;
        final AtomicArray<Object> reducedResponses;
        final AtomicReferenceArray<AtomicInteger> expectedOperationsPerItem;
        final AtomicReferenceArray<AtomicReferenceArray> responsesByItemAndShard;

        ASyncAction(MultiPercolateRequest multiPercolateRequest, List<Object> percolateRequests, ActionListener<MultiPercolateResponse> finalListener, ClusterState clusterState) {
            this.finalListener = finalListener;
            this.multiPercolateRequest = multiPercolateRequest;
            this.percolateRequests = percolateRequests;
            responsesByItemAndShard = new AtomicReferenceArray<>(percolateRequests.size());
            expectedOperationsPerItem = new AtomicReferenceArray<>(percolateRequests.size());
            reducedResponses = new AtomicArray<>(percolateRequests.size());

            // Resolving concrete indices and routing and grouping the requests by shard
            requestsByShard = new HashMap<>();
            // Keep track what slots belong to what shard, in case a request to a shard fails on all copies
            shardToSlots = new HashMap<>();
            int expectedResults = 0;
            for (int slot = 0;  slot < percolateRequests.size(); slot++) {
                Object element = percolateRequests.get(slot);
                assert element != null;
                if (element instanceof PercolateRequest) {
                    PercolateRequest percolateRequest = (PercolateRequest) element;
                    String[] concreteIndices;
                    try {
                         concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, percolateRequest);
                    } catch (IndexNotFoundException e) {
                        reducedResponses.set(slot, e);
                        responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
                        expectedOperationsPerItem.set(slot, new AtomicInteger(0));
                        continue;
                    }
                    Map<String, Set<String>> routing = indexNameExpressionResolver.resolveSearchRouting(clusterState, percolateRequest.routing(), percolateRequest.indices());
                    // TODO: I only need shardIds, ShardIterator(ShardRouting) is only needed in TransportShardMultiPercolateAction
                    GroupShardsIterator shards = clusterService.operationRouting().searchShards(
                            clusterState, concreteIndices, routing, percolateRequest.preference()
                    );
                    if (shards.size() == 0) {
                        reducedResponses.set(slot, new UnavailableShardsException(null, "No shards available"));
                        responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
                        expectedOperationsPerItem.set(slot, new AtomicInteger(0));
                        continue;
                    }

                    // The shard id is used as index in the atomic ref array, so we need to find out how many shards there are regardless of routing:
                    int numShards = clusterService.operationRouting().searchShardsCount(clusterState, concreteIndices, null);
                    responsesByItemAndShard.set(slot, new AtomicReferenceArray(numShards));
                    expectedOperationsPerItem.set(slot, new AtomicInteger(shards.size()));
                    for (ShardIterator shard : shards) {
                        ShardId shardId = shard.shardId();
                        TransportShardMultiPercolateAction.Request requests = requestsByShard.get(shardId);
                        if (requests == null) {
                            requestsByShard.put(shardId, requests = new TransportShardMultiPercolateAction.Request(shardId.getIndex(), shardId.getId(), percolateRequest.preference()));
                        }
                        logger.trace("Adding shard[{}] percolate request for item[{}]", shardId, slot);
                        requests.add(new TransportShardMultiPercolateAction.Request.Item(slot, new PercolateShardRequest(shardId, percolateRequest)));

                        IntArrayList items = shardToSlots.get(shardId);
                        if (items == null) {
                            shardToSlots.put(shardId, items = new IntArrayList());
                        }
                        items.add(slot);
                    }
                    expectedResults++;
                } else if (element instanceof Throwable || element instanceof MultiGetResponse.Failure) {
                    logger.trace("item[{}] won't be executed, reason: {}", slot, element);
                    reducedResponses.set(slot, element);
                    responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
                    expectedOperationsPerItem.set(slot, new AtomicInteger(0));
                }
            }
            expectedOperations = new AtomicInteger(expectedResults);
        }

        void run() {
            if (expectedOperations.get() == 0) {
                finish();
                return;
            }

            logger.trace("mpercolate executing for shards {}", requestsByShard.keySet());
            for (Map.Entry<ShardId, TransportShardMultiPercolateAction.Request> entry : requestsByShard.entrySet()) {
                final ShardId shardId = entry.getKey();
                TransportShardMultiPercolateAction.Request shardRequest = entry.getValue();
                shardMultiPercolateAction.execute(shardRequest, new ActionListener<TransportShardMultiPercolateAction.Response>() {

                    @Override
                    public void onResponse(TransportShardMultiPercolateAction.Response response) {
                        onShardResponse(shardId, response);
                    }

                    @Override
                    public void onFailure(Throwable e) {
                        onShardFailure(shardId, e);
                    }

                });
            }
        }

        @SuppressWarnings("unchecked")
        void onShardResponse(ShardId shardId, TransportShardMultiPercolateAction.Response response) {
            logger.trace("{} Percolate shard response", shardId);
            try {
                for (TransportShardMultiPercolateAction.Response.Item item : response.items()) {
                    AtomicReferenceArray shardResults = responsesByItemAndShard.get(item.slot());
                    if (shardResults == null) {
                        assert false : "shardResults can't be null";
                        continue;
                    }

                    if (item.failed()) {
                        shardResults.set(shardId.id(), new BroadcastShardOperationFailedException(shardId, item.error()));
                    } else {
                        shardResults.set(shardId.id(), item.response());
                    }

                    assert expectedOperationsPerItem.get(item.slot()).get() >= 1 : "slot[" + item.slot() + "] can't be lower than one";
                    if (expectedOperationsPerItem.get(item.slot()).decrementAndGet() == 0) {
                        // Failure won't bubble up, since we fail the whole request now via the catch clause below,
                        // so expectedOperationsPerItem will not be decremented twice.
                        reduce(item.slot());
                    }
                }
            } catch (Throwable e) {
                logger.error("{} Percolate original reduce error", e, shardId);
                finalListener.onFailure(e);
            }
        }

        @SuppressWarnings("unchecked")
        void onShardFailure(ShardId shardId, Throwable e) {
            logger.debug("{} Shard multi percolate failure", e, shardId);
            try {
                IntArrayList slots = shardToSlots.get(shardId);
                for (int i = 0; i < slots.size(); i++) {
                    int slot = slots.get(i);
                    AtomicReferenceArray shardResults = responsesByItemAndShard.get(slot);
                    if (shardResults == null) {
                        continue;
                    }

                    shardResults.set(shardId.id(), new BroadcastShardOperationFailedException(shardId, e));
                    assert expectedOperationsPerItem.get(slot).get() >= 1 : "slot[" + slot + "] can't be lower than one. Caused by: " + e.getMessage();
                    if (expectedOperationsPerItem.get(slot).decrementAndGet() == 0) {
                        reduce(slot);
                    }
                }
            } catch (Throwable t) {
                logger.error("{} Percolate original reduce error, original error {}", t, shardId, e);
                finalListener.onFailure(t);
            }
        }

        void reduce(int slot) {
            AtomicReferenceArray shardResponses = responsesByItemAndShard.get(slot);
            PercolateResponse reducedResponse = TransportPercolateAction.reduce((PercolateRequest) percolateRequests.get(slot), shardResponses, percolatorService);
            reducedResponses.set(slot, reducedResponse);
            assert expectedOperations.get() >= 1 : "slot[" + slot + "] expected options should be >= 1 but is " + expectedOperations.get();
            if (expectedOperations.decrementAndGet() == 0) {
                finish();
            }
        }

        void finish() {
            MultiPercolateResponse.Item[] finalResponse = new MultiPercolateResponse.Item[reducedResponses.length()];
            for (int slot = 0; slot < reducedResponses.length(); slot++) {
                Object element = reducedResponses.get(slot);
                assert element != null : "Element[" + slot + "] shouldn't be null";
                if (element instanceof PercolateResponse) {
                    finalResponse[slot] = new MultiPercolateResponse.Item((PercolateResponse) element);
                } else if (element instanceof Throwable) {
                    finalResponse[slot] = new MultiPercolateResponse.Item((Throwable)element);
                } else if (element instanceof MultiGetResponse.Failure) {
                    finalResponse[slot] = new MultiPercolateResponse.Item(((MultiGetResponse.Failure)element).getFailure());
                }
            }
            finalListener.onResponse(new MultiPercolateResponse(finalResponse));
        }

    }

}