/
HttpShardHandler.java
544 lines (469 loc) · 21.8 KB
/
HttpShardHandler.java
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
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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.apache.solr.handler.component;
import java.lang.invoke.MethodHandles;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.request.SolrQueryRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
public class HttpShardHandler extends ShardHandler {
/**
* If the request context map has an entry with this key and Boolean.TRUE as value,
* {@link #prepDistributed(ResponseBuilder)} will only include {@link org.apache.solr.common.cloud.Replica.Type#NRT} replicas as possible
* destination of the distributed request (or a leader replica of type {@link org.apache.solr.common.cloud.Replica.Type#TLOG}). This is used
* by the RealtimeGet handler, since other types of replicas shouldn't respond to RTG requests
*/
public static String ONLY_NRT_REPLICAS = "distribOnlyRealtime";
private HttpShardHandlerFactory httpShardHandlerFactory;
private CompletionService<ShardResponse> completionService;
private Set<Future<ShardResponse>> pending;
private Map<String,List<String>> shardToURLs;
private HttpClient httpClient;
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public HttpShardHandler(HttpShardHandlerFactory httpShardHandlerFactory, HttpClient httpClient) {
this.httpClient = httpClient;
this.httpShardHandlerFactory = httpShardHandlerFactory;
completionService = httpShardHandlerFactory.newCompletionService();
pending = new HashSet<>();
// maps "localhost:8983|localhost:7574" to a shuffled List("http://localhost:8983","http://localhost:7574")
// This is primarily to keep track of what order we should use to query the replicas of a shard
// so that we use the same replica for all phases of a distributed request.
shardToURLs = new HashMap<>();
}
private static class SimpleSolrResponse extends SolrResponse {
long elapsedTime;
NamedList<Object> nl;
@Override
public long getElapsedTime() {
return elapsedTime;
}
@Override
public NamedList<Object> getResponse() {
return nl;
}
@Override
public void setResponse(NamedList<Object> rsp) {
nl = rsp;
}
@Override
public void setElapsedTime(long elapsedTime) {
this.elapsedTime = elapsedTime;
}
}
// Not thread safe... don't use in Callable.
// Don't modify the returned URL list.
private List<String> getURLs(String shard) {
List<String> urls = shardToURLs.get(shard);
if (urls == null) {
urls = httpShardHandlerFactory.buildURLList(shard);
shardToURLs.put(shard, urls);
}
return urls;
}
@Override
public void submit(final ShardRequest sreq, final String shard, final ModifiableSolrParams params) {
// do this outside of the callable for thread safety reasons
final List<String> urls = getURLs(shard);
Callable<ShardResponse> task = () -> {
ShardResponse srsp = new ShardResponse();
if (sreq.nodeName != null) {
srsp.setNodeName(sreq.nodeName);
}
srsp.setShardRequest(sreq);
srsp.setShard(shard);
SimpleSolrResponse ssr = new SimpleSolrResponse();
srsp.setSolrResponse(ssr);
long startTime = System.nanoTime();
try {
params.remove(CommonParams.WT); // use default (currently javabin)
params.remove(CommonParams.VERSION);
QueryRequest req = makeQueryRequest(sreq, params, shard);
req.setMethod(SolrRequest.METHOD.POST);
// no need to set the response parser as binary is the default
// req.setResponseParser(new BinaryResponseParser());
// if there are no shards available for a slice, urls.size()==0
if (urls.size()==0) {
// TODO: what's the right error code here? We should use the same thing when
// all of the servers for a shard are down.
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "no servers hosting shard: " + shard);
}
if (urls.size() <= 1) {
String url = urls.get(0);
srsp.setShardAddress(url);
try (SolrClient client = new Builder(url).withHttpClient(httpClient).build()) {
ssr.nl = client.request(req);
}
} else {
LBHttpSolrClient.Rsp rsp = httpShardHandlerFactory.makeLoadBalancedRequest(req, urls);
ssr.nl = rsp.getResponse();
srsp.setShardAddress(rsp.getServer());
}
}
catch( ConnectException cex ) {
srsp.setException(cex); //????
} catch (Exception th) {
srsp.setException(th);
if (th instanceof SolrException) {
srsp.setResponseCode(((SolrException)th).code());
} else {
srsp.setResponseCode(-1);
}
}
ssr.elapsedTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
return transfomResponse(sreq, srsp, shard);
};
try {
if (shard != null) {
MDC.put("ShardRequest.shards", shard);
}
if (urls != null && !urls.isEmpty()) {
MDC.put("ShardRequest.urlList", urls.toString());
}
pending.add( completionService.submit(task) );
} finally {
MDC.remove("ShardRequest.shards");
MDC.remove("ShardRequest.urlList");
}
}
/**
* Subclasses could modify the request based on the shard
*/
protected QueryRequest makeQueryRequest(final ShardRequest sreq, ModifiableSolrParams params, String shard)
{
// use generic request to avoid extra processing of queries
return new QueryRequest(params);
}
/**
* Subclasses could modify the Response based on the the shard
*/
protected ShardResponse transfomResponse(final ShardRequest sreq, ShardResponse rsp, String shard)
{
return rsp;
}
/** returns a ShardResponse of the last response correlated with a ShardRequest. This won't
* return early if it runs into an error.
**/
@Override
public ShardResponse takeCompletedIncludingErrors() {
return take(false);
}
/** returns a ShardResponse of the last response correlated with a ShardRequest,
* or immediately returns a ShardResponse if there was an error detected
*/
@Override
public ShardResponse takeCompletedOrError() {
return take(true);
}
private ShardResponse take(boolean bailOnError) {
while (pending.size() > 0) {
try {
Future<ShardResponse> future = completionService.take();
pending.remove(future);
ShardResponse rsp = future.get();
if (bailOnError && rsp.getException() != null) return rsp; // if exception, return immediately
// add response to the response list... we do this after the take() and
// not after the completion of "call" so we know when the last response
// for a request was received. Otherwise we might return the same
// request more than once.
rsp.getShardRequest().responses.add(rsp);
if (rsp.getShardRequest().responses.size() == rsp.getShardRequest().actualShards.length) {
return rsp;
}
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
} catch (ExecutionException e) {
// should be impossible... the problem with catching the exception
// at this level is we don't know what ShardRequest it applied to
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Impossible Exception",e);
}
}
return null;
}
@Override
public void cancelAll() {
for (Future<ShardResponse> future : pending) {
future.cancel(false);
}
}
@Override
public void prepDistributed(ResponseBuilder rb) {
final SolrQueryRequest req = rb.req;
final SolrParams params = req.getParams();
final String shards = params.get(ShardParams.SHARDS);
// since the cost of grabbing cloud state is still up in the air, we grab it only
// if we need it.
ClusterState clusterState = null;
Map<String,Slice> slices = null;
CoreDescriptor coreDescriptor = req.getCore().getCoreDescriptor();
CloudDescriptor cloudDescriptor = coreDescriptor.getCloudDescriptor();
ZkController zkController = req.getCore().getCoreContainer().getZkController();
final ReplicaListTransformer replicaListTransformer = httpShardHandlerFactory.getReplicaListTransformer(req);
if (shards != null) {
List<String> lst = StrUtils.splitSmart(shards, ",", true);
rb.shards = lst.toArray(new String[lst.size()]);
rb.slices = new String[rb.shards.length];
if (zkController != null) {
// figure out which shards are slices
for (int i=0; i<rb.shards.length; i++) {
if (rb.shards[i].indexOf('/') < 0) {
// this is a logical shard
rb.slices[i] = rb.shards[i];
rb.shards[i] = null;
}
}
}
} else if (zkController != null) {
// we weren't provided with an explicit list of slices to query via "shards", so use the cluster state
clusterState = zkController.getClusterState();
String shardKeys = params.get(ShardParams._ROUTE_);
// This will be the complete list of slices we need to query for this request.
slices = new HashMap<>();
// we need to find out what collections this request is for.
// A comma-separated list of specified collections.
// Eg: "collection1,collection2,collection3"
String collections = params.get("collection");
if (collections != null) {
// If there were one or more collections specified in the query, split
// each parameter and store as a separate member of a List.
List<String> collectionList = StrUtils.splitSmart(collections, ",",
true);
// In turn, retrieve the slices that cover each collection from the
// cloud state and add them to the Map 'slices'.
for (String collectionName : collectionList) {
// The original code produced <collection-name>_<shard-name> when the collections
// parameter was specified (see ClientUtils.appendMap)
// Is this necessary if ony one collection is specified?
// i.e. should we change multiCollection to collectionList.size() > 1?
addSlices(slices, clusterState, params, collectionName, shardKeys, true);
}
} else {
// just this collection
String collectionName = cloudDescriptor.getCollectionName();
addSlices(slices, clusterState, params, collectionName, shardKeys, false);
}
// Store the logical slices in the ResponseBuilder and create a new
// String array to hold the physical shards (which will be mapped
// later).
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
}
HttpShardHandlerFactory.WhitelistHostChecker hostChecker = httpShardHandlerFactory.getWhitelistHostChecker();
if (shards != null && zkController == null && hostChecker.isWhitelistHostCheckingEnabled() && !hostChecker.hasExplicitWhitelist()) {
throw new SolrException(ErrorCode.FORBIDDEN, "HttpShardHandlerFactory "+HttpShardHandlerFactory.INIT_SHARDS_WHITELIST
+" not configured but required (in lieu of ZkController and ClusterState) when using the '"+ShardParams.SHARDS+"' parameter."
+HttpShardHandlerFactory.SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE);
}
//
// Map slices to shards
//
if (zkController != null) {
// Are we hosting the shard that this request is for, and are we active? If so, then handle it ourselves
// and make it a non-distributed request.
String ourSlice = cloudDescriptor.getShardId();
String ourCollection = cloudDescriptor.getCollectionName();
// Some requests may only be fulfilled by replicas of type Replica.Type.NRT
boolean onlyNrtReplicas = Boolean.TRUE == req.getContext().get(ONLY_NRT_REPLICAS);
if (rb.slices.length == 1 && rb.slices[0] != null
&& ( rb.slices[0].equals(ourSlice) || rb.slices[0].equals(ourCollection + "_" + ourSlice) ) // handle the <collection>_<slice> format
&& cloudDescriptor.getLastPublished() == Replica.State.ACTIVE
&& (!onlyNrtReplicas || cloudDescriptor.getReplicaType() == Replica.Type.NRT)) {
boolean shortCircuit = params.getBool("shortCircuit", true); // currently just a debugging parameter to check distrib search on a single node
String targetHandler = params.get(ShardParams.SHARDS_QT);
shortCircuit = shortCircuit && targetHandler == null; // if a different handler is specified, don't short-circuit
if (shortCircuit) {
rb.isDistrib = false;
rb.shortCircuitedURL = ZkCoreNodeProps.getCoreUrl(zkController.getBaseUrl(), coreDescriptor.getName());
if (hostChecker.isWhitelistHostCheckingEnabled() && hostChecker.hasExplicitWhitelist()) {
/*
* We only need to check the host whitelist if there is an explicit whitelist (other than all the live nodes)
* when the "shards" indicate cluster state elements only
*/
hostChecker.checkWhitelist(clusterState, shards, Arrays.asList(rb.shortCircuitedURL));
}
return;
}
// We shouldn't need to do anything to handle "shard.rows" since it was previously meant to be an optimization?
}
if (clusterState == null && zkController != null) {
clusterState = zkController.getClusterState();
}
for (int i=0; i<rb.shards.length; i++) {
if (rb.shards[i] != null) {
final List<String> shardUrls = StrUtils.splitSmart(rb.shards[i], "|", true);
replicaListTransformer.transform(shardUrls);
hostChecker.checkWhitelist(clusterState, shards, shardUrls);
// And now recreate the | delimited list of equivalent servers
rb.shards[i] = createSliceShardsStr(shardUrls);
} else {
if (slices == null) {
slices = clusterState.getCollection(cloudDescriptor.getCollectionName()).getSlicesMap();
}
String sliceName = rb.slices[i];
Slice slice = slices.get(sliceName);
if (slice==null) {
// Treat this the same as "all servers down" for a slice, and let things continue
// if partial results are acceptable
rb.shards[i] = "";
continue;
// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such shard: " + sliceName);
}
final Predicate<Replica> isShardLeader = new Predicate<Replica>() {
private Replica shardLeader = null;
@Override
public boolean test(Replica replica) {
if (shardLeader == null) {
try {
shardLeader = zkController.getZkStateReader().getLeaderRetry(cloudDescriptor.getCollectionName(), slice.getName());
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Exception finding leader for shard " + slice.getName() + " in collection "
+ cloudDescriptor.getCollectionName(), e);
} catch (SolrException e) {
if (log.isDebugEnabled()) {
log.debug("Exception finding leader for shard {} in collection {}. Collection State: {}",
slice.getName(), cloudDescriptor.getCollectionName(), zkController.getZkStateReader().getClusterState().getCollectionOrNull(cloudDescriptor.getCollectionName()));
}
throw e;
}
}
return replica.getName().equals(shardLeader.getName());
}
};
final List<Replica> eligibleSliceReplicas = collectEligibleReplicas(slice, clusterState, onlyNrtReplicas, isShardLeader);
final List<String> shardUrls = transformReplicasToShardUrls(replicaListTransformer, eligibleSliceReplicas);
if (hostChecker.isWhitelistHostCheckingEnabled() && hostChecker.hasExplicitWhitelist()) {
/*
* We only need to check the host whitelist if there is an explicit whitelist (other than all the live nodes)
* when the "shards" indicate cluster state elements only
*/
hostChecker.checkWhitelist(clusterState, shards, shardUrls);
}
// And now recreate the | delimited list of equivalent servers
final String sliceShardsStr = createSliceShardsStr(shardUrls);
if (sliceShardsStr.isEmpty()) {
boolean tolerant = ShardParams.getShardsTolerantAsBool(rb.req.getParams());
if (!tolerant) {
// stop the check when there are no replicas available for a shard
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
"no servers hosting shard: " + rb.slices[i]);
}
}
rb.shards[i] = sliceShardsStr;
}
}
} else {
if (shards != null) {
// No cloud, verbatim check of shards
hostChecker.checkWhitelist(shards, new ArrayList<>(Arrays.asList(shards.split("[,|]"))));
}
}
String shards_rows = params.get(ShardParams.SHARDS_ROWS);
if(shards_rows != null) {
rb.shards_rows = Integer.parseInt(shards_rows);
}
String shards_start = params.get(ShardParams.SHARDS_START);
if(shards_start != null) {
rb.shards_start = Integer.parseInt(shards_start);
}
}
private static List<Replica> collectEligibleReplicas(Slice slice, ClusterState clusterState, boolean onlyNrtReplicas, Predicate<Replica> isShardLeader) {
final Collection<Replica> allSliceReplicas = slice.getReplicasMap().values();
final List<Replica> eligibleSliceReplicas = new ArrayList<>(allSliceReplicas.size());
for (Replica replica : allSliceReplicas) {
if (!clusterState.liveNodesContain(replica.getNodeName())
|| replica.getState() != Replica.State.ACTIVE
|| (onlyNrtReplicas && replica.getType() == Replica.Type.PULL)) {
continue;
}
if (onlyNrtReplicas && replica.getType() == Replica.Type.TLOG) {
if (!isShardLeader.test(replica)) {
continue;
}
}
eligibleSliceReplicas.add(replica);
}
return eligibleSliceReplicas;
}
private static List<String> transformReplicasToShardUrls(final ReplicaListTransformer replicaListTransformer, final List<Replica> eligibleSliceReplicas) {
replicaListTransformer.transform(eligibleSliceReplicas);
final List<String> shardUrls = new ArrayList<>(eligibleSliceReplicas.size());
for (Replica replica : eligibleSliceReplicas) {
String url = ZkCoreNodeProps.getCoreUrl(replica);
shardUrls.add(url);
}
return shardUrls;
}
private static String createSliceShardsStr(final List<String> shardUrls) {
final StringBuilder sliceShardsStr = new StringBuilder();
boolean first = true;
for (String shardUrl : shardUrls) {
if (first) {
first = false;
} else {
sliceShardsStr.append('|');
}
sliceShardsStr.append(shardUrl);
}
return sliceShardsStr.toString();
}
private void addSlices(Map<String,Slice> target, ClusterState state, SolrParams params, String collectionName, String shardKeys, boolean multiCollection) {
DocCollection coll = state.getCollection(collectionName);
Collection<Slice> slices = coll.getRouter().getSearchSlices(shardKeys, params , coll);
ClientUtils.addSlices(target, collectionName, slices, multiCollection);
}
public ShardHandlerFactory getShardHandlerFactory(){
return httpShardHandlerFactory;
}
}