forked from apache/solr
-
Notifications
You must be signed in to change notification settings - Fork 1
/
CloudSolrClient.java
1387 lines (1237 loc) · 52.6 KB
/
CloudSolrClient.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
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.client.solrj.impl;
import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
import static org.apache.solr.common.params.CommonParams.ID;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.ConnectException;
import java.net.SocketException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.ResponseParser;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.V2RequestSupport;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.IsUpdateRequest;
import org.apache.solr.client.solrj.request.RequestWriter;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.client.solrj.routing.ReplicaListTransformer;
import org.apache.solr.client.solrj.routing.RequestReplicaListTransformerGenerator;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.ToleratedUpdateError;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.ImplicitDocRouter;
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.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.Hash;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
public abstract class CloudSolrClient extends SolrClient {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private volatile String defaultCollection;
// no of times collection state to be reloaded if stale state error is received
private static final int MAX_STALE_RETRIES =
Integer.parseInt(System.getProperty("cloudSolrClientMaxStaleRetries", "5"));
private final Random rand = new Random();
private final boolean updatesToLeaders;
private final boolean directUpdatesToLeadersOnly;
private final RequestReplicaListTransformerGenerator requestRLTGenerator;
boolean parallelUpdates; // TODO final
private ExecutorService threadPool =
ExecutorUtil.newMDCAwareCachedThreadPool(
new SolrNamedThreadFactory("CloudSolrClient ThreadPool"));
public static final String STATE_VERSION = "_stateVer_";
protected long retryExpiryTime =
TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS); // 3 seconds or 3 million nanos
private final Set<String> NON_ROUTABLE_PARAMS;
{
NON_ROUTABLE_PARAMS = new HashSet<>();
NON_ROUTABLE_PARAMS.add(UpdateParams.EXPUNGE_DELETES);
NON_ROUTABLE_PARAMS.add(UpdateParams.MAX_OPTIMIZE_SEGMENTS);
NON_ROUTABLE_PARAMS.add(UpdateParams.COMMIT);
NON_ROUTABLE_PARAMS.add(UpdateParams.WAIT_SEARCHER);
NON_ROUTABLE_PARAMS.add(UpdateParams.OPEN_SEARCHER);
NON_ROUTABLE_PARAMS.add(UpdateParams.SOFT_COMMIT);
NON_ROUTABLE_PARAMS.add(UpdateParams.PREPARE_COMMIT);
NON_ROUTABLE_PARAMS.add(UpdateParams.OPTIMIZE);
// Not supported via SolrCloud
// NON_ROUTABLE_PARAMS.add(UpdateParams.ROLLBACK);
}
private volatile List<Object> locks = objectList(3);
/** Constructs {@link CloudSolrClient} instances from provided configuration. */
public static class Builder extends CloudHttp2SolrClient.Builder {
/**
* Provide a series of Solr URLs to be used when configuring {@link CloudSolrClient} instances.
* The solr client will use these urls to understand the cluster topology, which solr nodes are
* active etc.
*
* <p>Provided Solr URLs are expected to point to the root Solr path
* ("http://hostname:8983/solr"); they should not include any collections, cores, or other path
* components.
*
* <p>Usage example:
*
* <pre>
* final List<String> solrBaseUrls = new ArrayList<String>();
* solrBaseUrls.add("http://solr1:8983/solr"); solrBaseUrls.add("http://solr2:8983/solr"); solrBaseUrls.add("http://solr3:8983/solr");
* final SolrClient client = new CloudSolrClient.Builder(solrBaseUrls).build();
* </pre>
*/
public Builder(List<String> solrUrls) {
super(solrUrls);
}
/**
* Provide a series of ZK hosts which will be used when configuring {@link CloudSolrClient}
* instances. This requires a dependency on {@code solr-solrj-zookeeper} which transitively
* depends on more JARs. The ZooKeeper based connection is the most reliable and performant
* means for CloudSolrClient to work. On the other hand, it means exposing ZooKeeper more
* broadly than to Solr nodes, which is a security risk.
*
* <p>Usage example when Solr stores data at the ZooKeeper root ('/'):
*
* <pre>
* final List<String> zkServers = new ArrayList<String>();
* zkServers.add("zookeeper1:2181"); zkServers.add("zookeeper2:2181"); zkServers.add("zookeeper3:2181");
* final SolrClient client = new CloudSolrClient.Builder(zkServers, Optional.empty()).build();
* </pre>
*
* Usage example when Solr data is stored in a ZooKeeper chroot:
*
* <pre>
* final List<String> zkServers = new ArrayList<String>();
* zkServers.add("zookeeper1:2181"); zkServers.add("zookeeper2:2181"); zkServers.add("zookeeper3:2181");
* final SolrClient client = new CloudSolrClient.Builder(zkServers, Optional.of("/solr")).build();
* </pre>
*
* @param zkHosts a List of at least one ZooKeeper host and port (e.g. "zookeeper1:2181")
* @param zkChroot the path to the root ZooKeeper node containing Solr data. Provide {@code
* java.util.Optional.empty()} if no ZK chroot is used.
*/
public Builder(List<String> zkHosts, Optional<String> zkChroot) {
super(zkHosts, zkChroot);
}
}
static class StateCache extends ConcurrentHashMap<String, ExpiringCachedDocCollection> {
final AtomicLong puts = new AtomicLong();
final AtomicLong hits = new AtomicLong();
final Lock evictLock = new ReentrantLock(true);
protected volatile long timeToLive = 60 * 1000L;
@Override
public ExpiringCachedDocCollection get(Object key) {
ExpiringCachedDocCollection val = super.get(key);
if (val == null) {
// a new collection is likely to be added now.
// check if there are stale items and remove them
evictStale();
return null;
}
if (val.isExpired(timeToLive)) {
super.remove(key);
return null;
}
hits.incrementAndGet();
return val;
}
@Override
public ExpiringCachedDocCollection put(String key, ExpiringCachedDocCollection value) {
puts.incrementAndGet();
return super.put(key, value);
}
void evictStale() {
if (!evictLock.tryLock()) return;
try {
for (Entry<String, ExpiringCachedDocCollection> e : entrySet()) {
if (e.getValue().isExpired(timeToLive)) {
super.remove(e.getKey());
}
}
} finally {
evictLock.unlock();
}
}
}
/**
* This is the time to wait to refetch the state after getting the same state version from ZK
*
* <p>secs
*
* @deprecated use {@link CloudSolrClient.Builder#setRetryExpiryTime(int)} instead
*/
@Deprecated
public void setRetryExpiryTime(int secs) {
this.retryExpiryTime = TimeUnit.NANOSECONDS.convert(secs, TimeUnit.SECONDS);
}
protected final StateCache collectionStateCache = new StateCache();
class ExpiringCachedDocCollection {
final DocCollection cached;
final long cachedAt;
// This is the time at which the collection is retried and got the same old version
volatile long retriedAt = -1;
// flag that suggests that this is potentially to be rechecked
volatile boolean maybeStale = false;
ExpiringCachedDocCollection(DocCollection cached) {
this.cached = cached;
this.cachedAt = System.nanoTime();
}
boolean isExpired(long timeToLiveMs) {
return (System.nanoTime() - cachedAt)
> TimeUnit.NANOSECONDS.convert(timeToLiveMs, TimeUnit.MILLISECONDS);
}
boolean shouldRetry() {
if (maybeStale) { // we are not sure if it is stale so check with retry time
if ((retriedAt == -1 || (System.nanoTime() - retriedAt) > retryExpiryTime)) {
return true; // we retried a while back. and we could not get anything new.
// it's likely that it is not going to be available now also.
}
}
return false;
}
void setRetriedAt() {
retriedAt = System.nanoTime();
}
}
protected CloudSolrClient(
boolean updatesToLeaders, boolean parallelUpdates, boolean directUpdatesToLeadersOnly) {
this.updatesToLeaders = updatesToLeaders;
this.parallelUpdates = parallelUpdates;
this.directUpdatesToLeadersOnly = directUpdatesToLeadersOnly;
this.requestRLTGenerator = new RequestReplicaListTransformerGenerator();
}
/**
* Sets the cache ttl for DocCollection Objects cached.
*
* @param seconds ttl value in seconds
*/
public void setCollectionCacheTTl(int seconds) {
assert seconds > 0;
this.collectionStateCache.timeToLive = seconds * 1000L;
}
protected abstract LBSolrClient getLbClient();
public abstract ClusterStateProvider getClusterStateProvider();
public ClusterState getClusterState() {
return getClusterStateProvider().getClusterState();
}
protected abstract boolean wasCommError(Throwable t);
@Override
public void close() throws IOException {
if (this.threadPool != null && !ExecutorUtil.isShutdown(this.threadPool)) {
ExecutorUtil.shutdownAndAwaitTermination(this.threadPool);
this.threadPool = null;
}
}
public ResponseParser getParser() {
return getLbClient().getParser();
}
/**
* Note: This setter method is <b>not thread-safe</b>.
*
* @param processor Default Response Parser chosen to parse the response if the parser were not
* specified as part of the request.
* @see org.apache.solr.client.solrj.SolrRequest#getResponseParser()
* @deprecated use {@link CloudHttp2SolrClient.Builder} instead
*/
@Deprecated
public void setParser(ResponseParser processor) {
getLbClient().setParser(processor);
}
public RequestWriter getRequestWriter() {
return getLbClient().getRequestWriter();
}
/**
* Choose the {@link RequestWriter} to use.
*
* <p>Note: This setter method is <b>not thread-safe</b>.
*
* @deprecated use {@link CloudHttp2SolrClient.Builder} instead
*/
@Deprecated
public void setRequestWriter(RequestWriter requestWriter) {
getLbClient().setRequestWriter(requestWriter);
}
/** Sets the default collection for request */
public void setDefaultCollection(String collection) {
this.defaultCollection = collection;
}
/** Gets the default collection for request */
public String getDefaultCollection() {
return defaultCollection;
}
/** Gets whether direct updates are sent in parallel */
public boolean isParallelUpdates() {
return parallelUpdates;
}
/**
* Connect to the zookeeper ensemble. This is an optional method that may be used to force a
* connect before any other requests are sent.
*/
public void connect() {
getClusterStateProvider().connect();
}
/**
* Connect to a cluster. If the cluster is not ready, retry connection up to a given timeout.
*
* @param duration the timeout
* @param timeUnit the units of the timeout
* @throws TimeoutException if the cluster is not ready after the timeout
* @throws InterruptedException if the wait is interrupted
*/
public void connect(long duration, TimeUnit timeUnit)
throws TimeoutException, InterruptedException {
if (log.isInfoEnabled()) {
log.info(
"Waiting for {} {} for cluster at {} to be ready",
duration,
timeUnit,
getClusterStateProvider());
}
long timeout = System.nanoTime() + timeUnit.toNanos(duration);
while (System.nanoTime() < timeout) {
try {
connect();
if (log.isInfoEnabled()) {
log.info("Cluster at {} ready", getClusterStateProvider());
}
return;
} catch (RuntimeException e) {
// not ready yet, then...
}
TimeUnit.MILLISECONDS.sleep(250);
}
throw new TimeoutException("Timed out waiting for cluster");
}
@SuppressWarnings({"unchecked"})
private NamedList<Object> directUpdate(AbstractUpdateRequest request, String collection)
throws SolrServerException {
UpdateRequest updateRequest = (UpdateRequest) request;
SolrParams params = request.getParams();
ModifiableSolrParams routableParams = new ModifiableSolrParams();
ModifiableSolrParams nonRoutableParams = new ModifiableSolrParams();
if (params != null) {
nonRoutableParams.add(params);
routableParams.add(params);
for (String param : NON_ROUTABLE_PARAMS) {
routableParams.remove(param);
}
} else {
params = new ModifiableSolrParams();
}
if (collection == null) {
throw new SolrServerException(
"No collection param specified on request and no default collection has been set.");
}
// Check to see if the collection is an alias. Updates to multi-collection aliases are ok as
// long as they are routed aliases
List<String> aliasedCollections = getClusterStateProvider().resolveAlias(collection);
if (getClusterStateProvider().isRoutedAlias(collection) || aliasedCollections.size() == 1) {
collection = aliasedCollections.get(0); // pick 1st (consistent with HttpSolrCall behavior)
} else {
throw new SolrException(
SolrException.ErrorCode.BAD_REQUEST,
"Update request to non-routed multi-collection alias not supported: "
+ collection
+ " -> "
+ aliasedCollections);
}
DocCollection col = getDocCollection(collection, null);
DocRouter router = col.getRouter();
if (router instanceof ImplicitDocRouter) {
// short circuit as optimization
return null;
}
ReplicaListTransformer replicaListTransformer =
requestRLTGenerator.getReplicaListTransformer(params);
// Create the URL map, which is keyed on slice name.
// The value is a list of URLs for each replica in the slice.
// The first value in the list is the leader for the slice.
final Map<String, List<String>> urlMap = buildUrlMap(col, replicaListTransformer);
String routeField =
(col.getRouter().getRouteField(col) == null) ? ID : col.getRouter().getRouteField(col);
final Map<String, ? extends LBSolrClient.Req> routes =
createRoutes(updateRequest, routableParams, col, router, urlMap, routeField);
if (routes == null) {
if (directUpdatesToLeadersOnly && hasInfoToFindLeaders(updateRequest, routeField)) {
// we have info (documents with ids and/or ids to delete) with
// which to find the leaders but we could not find (all of) them
throw new SolrException(
SolrException.ErrorCode.SERVICE_UNAVAILABLE,
"directUpdatesToLeadersOnly==true but could not find leader(s)");
} else {
// we could not find a leader or routes yet - use unoptimized general path
return null;
}
}
final NamedList<Throwable> exceptions = new NamedList<>();
final NamedList<NamedList<?>> shardResponses =
new NamedList<>(routes.size() + 1); // +1 for deleteQuery
long start = System.nanoTime();
if (parallelUpdates) {
final Map<String, Future<NamedList<?>>> responseFutures = new HashMap<>(routes.size());
for (final Map.Entry<String, ? extends LBSolrClient.Req> entry : routes.entrySet()) {
final String url = entry.getKey();
final LBSolrClient.Req lbRequest = entry.getValue();
try {
MDC.put("CloudSolrClient.url", url);
responseFutures.put(
url,
threadPool.submit(
() -> {
return getLbClient().request(lbRequest).getResponse();
}));
} finally {
MDC.remove("CloudSolrClient.url");
}
}
for (final Map.Entry<String, Future<NamedList<?>>> entry : responseFutures.entrySet()) {
final String url = entry.getKey();
final Future<NamedList<?>> responseFuture = entry.getValue();
try {
shardResponses.add(url, responseFuture.get());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
} catch (ExecutionException e) {
exceptions.add(url, e.getCause());
}
}
if (exceptions.size() > 0) {
Throwable firstException = exceptions.getVal(0);
if (firstException instanceof SolrException) {
SolrException e = (SolrException) firstException;
throw getRouteException(
SolrException.ErrorCode.getErrorCode(e.code()), exceptions, routes);
} else {
throw getRouteException(SolrException.ErrorCode.SERVER_ERROR, exceptions, routes);
}
}
} else {
for (Map.Entry<String, ? extends LBSolrClient.Req> entry : routes.entrySet()) {
String url = entry.getKey();
LBSolrClient.Req lbRequest = entry.getValue();
try {
NamedList<Object> rsp = getLbClient().request(lbRequest).getResponse();
shardResponses.add(url, rsp);
} catch (Exception e) {
if (e instanceof SolrException) {
throw (SolrException) e;
} else {
throw new SolrServerException(e);
}
}
}
}
UpdateRequest nonRoutableRequest = null;
List<String> deleteQuery = updateRequest.getDeleteQuery();
if (deleteQuery != null && deleteQuery.size() > 0) {
UpdateRequest deleteQueryRequest = new UpdateRequest();
deleteQueryRequest.setDeleteQuery(deleteQuery);
nonRoutableRequest = deleteQueryRequest;
}
Set<String> paramNames = nonRoutableParams.getParameterNames();
Set<String> intersection = new HashSet<>(paramNames);
intersection.retainAll(NON_ROUTABLE_PARAMS);
if (nonRoutableRequest != null || intersection.size() > 0) {
if (nonRoutableRequest == null) {
nonRoutableRequest = new UpdateRequest();
}
nonRoutableRequest.setParams(nonRoutableParams);
nonRoutableRequest.setBasicAuthCredentials(
request.getBasicAuthUser(), request.getBasicAuthPassword());
List<String> urlList = new ArrayList<>(routes.keySet());
Collections.shuffle(urlList, rand);
LBSolrClient.Req req = new LBSolrClient.Req(nonRoutableRequest, urlList);
try {
LBSolrClient.Rsp rsp = getLbClient().request(req);
shardResponses.add(urlList.get(0), rsp.getResponse());
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, urlList.get(0), e);
}
}
long end = System.nanoTime();
@SuppressWarnings({"rawtypes"})
RouteResponse rr =
condenseResponse(
shardResponses, (int) TimeUnit.MILLISECONDS.convert(end - start, TimeUnit.NANOSECONDS));
rr.setRouteResponses(shardResponses);
rr.setRoutes(routes);
return rr;
}
protected RouteException getRouteException(
SolrException.ErrorCode serverError,
NamedList<Throwable> exceptions,
Map<String, ? extends LBSolrClient.Req> routes) {
return new RouteException(serverError, exceptions, routes);
}
protected Map<String, ? extends LBSolrClient.Req> createRoutes(
UpdateRequest updateRequest,
ModifiableSolrParams routableParams,
DocCollection col,
DocRouter router,
Map<String, List<String>> urlMap,
String routeField) {
return urlMap == null
? null
: updateRequest.getRoutesToCollection(router, col, urlMap, routableParams, routeField);
}
private Map<String, List<String>> buildUrlMap(
DocCollection col, ReplicaListTransformer replicaListTransformer) {
Map<String, List<String>> urlMap = new HashMap<>();
Slice[] slices = col.getActiveSlicesArr();
for (Slice slice : slices) {
String name = slice.getName();
List<Replica> sortedReplicas = new ArrayList<>();
Replica leader = slice.getLeader();
if (directUpdatesToLeadersOnly && leader == null) {
for (Replica replica :
slice.getReplicas(
replica ->
replica.isActive(getClusterStateProvider().getLiveNodes())
&& replica.getType() == Replica.Type.NRT)) {
leader = replica;
break;
}
}
if (leader == null) {
if (directUpdatesToLeadersOnly) {
continue;
}
// take unoptimized general path - we cannot find a leader yet
return null;
}
if (!directUpdatesToLeadersOnly) {
for (Replica replica : slice.getReplicas()) {
if (!replica.equals(leader)) {
sortedReplicas.add(replica);
}
}
}
// Sort the non-leader replicas according to the request parameters
replicaListTransformer.transform(sortedReplicas);
// put the leaderUrl first.
sortedReplicas.add(0, leader);
urlMap.put(
name, sortedReplicas.stream().map(Replica::getCoreUrl).collect(Collectors.toList()));
}
return urlMap;
}
protected <T extends RouteResponse<?>> T condenseResponse(
NamedList<?> response, int timeMillis, Supplier<T> supplier) {
T condensed = supplier.get();
int status = 0;
Integer rf = null;
// TolerantUpdateProcessor
List<SimpleOrderedMap<String>> toleratedErrors = null;
int maxToleratedErrors = Integer.MAX_VALUE;
// For "adds", "deletes", "deleteByQuery" etc.
Map<String, NamedList<Object>> versions = new HashMap<>();
for (int i = 0; i < response.size(); i++) {
NamedList<?> shardResponse = (NamedList<?>) response.getVal(i);
NamedList<?> header = (NamedList<?>) shardResponse.get("responseHeader");
Integer shardStatus = (Integer) header.get("status");
int s = shardStatus.intValue();
if (s > 0) {
status = s;
}
Object rfObj = header.get(UpdateRequest.REPFACT);
if (rfObj != null && rfObj instanceof Integer) {
Integer routeRf = (Integer) rfObj;
if (rf == null || routeRf < rf) rf = routeRf;
}
@SuppressWarnings("unchecked")
List<SimpleOrderedMap<String>> shardTolerantErrors =
(List<SimpleOrderedMap<String>>) header.get("errors");
if (null != shardTolerantErrors) {
Integer shardMaxToleratedErrors = (Integer) header.get("maxErrors");
assert null != shardMaxToleratedErrors
: "TolerantUpdateProcessor reported errors but not maxErrors";
// if we get into some weird state where the nodes disagree about the effective maxErrors,
// assume the min value seen to decide if we should fail.
maxToleratedErrors =
Math.min(
maxToleratedErrors,
ToleratedUpdateError.getEffectiveMaxErrors(shardMaxToleratedErrors.intValue()));
if (null == toleratedErrors) {
toleratedErrors = new ArrayList<SimpleOrderedMap<String>>(shardTolerantErrors.size());
}
for (SimpleOrderedMap<String> err : shardTolerantErrors) {
toleratedErrors.add(err);
}
}
for (String updateType : Arrays.asList("adds", "deletes", "deleteByQuery")) {
Object obj = shardResponse.get(updateType);
if (obj instanceof NamedList) {
NamedList<Object> versionsList =
versions.containsKey(updateType) ? versions.get(updateType) : new NamedList<>();
NamedList<?> nl = (NamedList<?>) obj;
versionsList.addAll(nl);
versions.put(updateType, versionsList);
}
}
}
NamedList<Object> cheader = new NamedList<>();
cheader.add("status", status);
cheader.add("QTime", timeMillis);
if (rf != null) cheader.add(UpdateRequest.REPFACT, rf);
if (null != toleratedErrors) {
cheader.add("maxErrors", ToleratedUpdateError.getUserFriendlyMaxErrors(maxToleratedErrors));
cheader.add("errors", toleratedErrors);
if (maxToleratedErrors < toleratedErrors.size()) {
// cumulative errors are too high, we need to throw a client exception w/correct metadata
// NOTE: it shouldn't be possible for 1 == toleratedErrors.size(), because if that were the
// case then at least one shard should have thrown a real error before this, so we don't
// worry about having a more "singular" exception msg for that situation
StringBuilder msgBuf =
new StringBuilder()
.append(toleratedErrors.size())
.append(" Async failures during distributed update: ");
NamedList<String> metadata = new NamedList<>();
for (SimpleOrderedMap<String> err : toleratedErrors) {
ToleratedUpdateError te = ToleratedUpdateError.parseMap(err);
metadata.add(te.getMetadataKey(), te.getMetadataValue());
msgBuf.append("\n").append(te.getMessage());
}
SolrException toThrow =
new SolrException(SolrException.ErrorCode.BAD_REQUEST, msgBuf.toString());
toThrow.setMetadata(metadata);
throw toThrow;
}
}
for (Map.Entry<String, NamedList<Object>> entry : versions.entrySet()) {
condensed.add(entry.getKey(), entry.getValue());
}
condensed.add("responseHeader", cheader);
return condensed;
}
@SuppressWarnings({"rawtypes"})
public RouteResponse condenseResponse(NamedList<?> response, int timeMillis) {
return condenseResponse(response, timeMillis, RouteResponse::new);
}
@SuppressWarnings({"rawtypes"})
public static class RouteResponse<T extends LBSolrClient.Req> extends NamedList<Object> {
private NamedList<NamedList<?>> routeResponses;
private Map<String, T> routes;
public void setRouteResponses(NamedList<NamedList<?>> routeResponses) {
this.routeResponses = routeResponses;
}
public NamedList<NamedList<?>> getRouteResponses() {
return routeResponses;
}
public void setRoutes(Map<String, T> routes) {
this.routes = routes;
}
public Map<String, T> getRoutes() {
return routes;
}
}
public static class RouteException extends SolrException {
private NamedList<Throwable> throwables;
private Map<String, ? extends LBSolrClient.Req> routes;
public RouteException(
ErrorCode errorCode,
NamedList<Throwable> throwables,
Map<String, ? extends LBSolrClient.Req> routes) {
super(errorCode, throwables.getVal(0).getMessage(), throwables.getVal(0));
this.throwables = throwables;
this.routes = routes;
// create a merged copy of the metadata from all wrapped exceptions
NamedList<String> metadata = new NamedList<String>();
for (int i = 0; i < throwables.size(); i++) {
Throwable t = throwables.getVal(i);
if (t instanceof SolrException) {
SolrException e = (SolrException) t;
NamedList<String> eMeta = e.getMetadata();
if (null != eMeta) {
metadata.addAll(eMeta);
}
}
}
if (0 < metadata.size()) {
this.setMetadata(metadata);
}
}
public NamedList<Throwable> getThrowables() {
return throwables;
}
public Map<String, ? extends LBSolrClient.Req> getRoutes() {
return this.routes;
}
}
@Override
public NamedList<Object> request(SolrRequest<?> request, String collection)
throws SolrServerException, IOException {
// the collection parameter of the request overrides that of the parameter to this method
String requestCollection = request.getCollection();
if (requestCollection != null) {
collection = requestCollection;
} else if (collection == null) {
collection = defaultCollection;
}
List<String> inputCollections =
collection == null ? Collections.emptyList() : StrUtils.splitSmart(collection, ",", true);
return requestWithRetryOnStaleState(request, 0, inputCollections);
}
/**
* As this class doesn't watch external collections on the client side, there's a chance that the
* request will fail due to cached stale state, which means the state must be refreshed from ZK
* and retried.
*/
protected NamedList<Object> requestWithRetryOnStaleState(
SolrRequest<?> request, int retryCount, List<String> inputCollections)
throws SolrServerException, IOException {
connect(); // important to call this before you start working with the ZkStateReader
// build up a _stateVer_ param to pass to the server containing all of the
// external collection state versions involved in this request, which allows
// the server to notify us that our cached state for one or more of the external
// collections is stale and needs to be refreshed ... this code has no impact on internal
// collections
String stateVerParam = null;
List<DocCollection> requestedCollections = null;
boolean isCollectionRequestOfV2 = false;
if (request instanceof V2RequestSupport) {
request = ((V2RequestSupport) request).getV2Request();
}
if (request instanceof V2Request) {
isCollectionRequestOfV2 = ((V2Request) request).isPerCollectionRequest();
}
boolean isAdmin = ADMIN_PATHS.contains(request.getPath());
boolean isUpdate = (request instanceof IsUpdateRequest) && (request instanceof UpdateRequest);
if (!inputCollections.isEmpty()
&& !isAdmin
&& !isCollectionRequestOfV2) { // don't do _stateVer_ checking for admin, v2 api requests
Set<String> requestedCollectionNames = resolveAliases(inputCollections, isUpdate);
StringBuilder stateVerParamBuilder = null;
for (String requestedCollection : requestedCollectionNames) {
// track the version of state we're using on the client side using the _stateVer_ param
DocCollection coll = getDocCollection(requestedCollection, null);
if (coll == null) {
throw new SolrException(
SolrException.ErrorCode.BAD_REQUEST, "Collection not found: " + requestedCollection);
}
int collVer = coll.getZNodeVersion();
if (requestedCollections == null)
requestedCollections = new ArrayList<>(requestedCollectionNames.size());
requestedCollections.add(coll);
if (stateVerParamBuilder == null) {
stateVerParamBuilder = new StringBuilder();
} else {
stateVerParamBuilder.append(
"|"); // hopefully pipe is not an allowed char in a collection name
}
stateVerParamBuilder.append(coll.getName()).append(":").append(collVer);
}
if (stateVerParamBuilder != null) {
stateVerParam = stateVerParamBuilder.toString();
}
}
if (request.getParams() instanceof ModifiableSolrParams) {
ModifiableSolrParams params = (ModifiableSolrParams) request.getParams();
if (stateVerParam != null) {
params.set(STATE_VERSION, stateVerParam);
} else {
params.remove(STATE_VERSION);
}
} // else: ??? how to set this ???
NamedList<Object> resp = null;
try {
resp = sendRequest(request, inputCollections);
// to avoid an O(n) operation we always add STATE_VERSION to the last and try to read it from
// there
Object o = resp == null || resp.size() == 0 ? null : resp.get(STATE_VERSION, resp.size() - 1);
if (o != null && o instanceof Map) {
// remove this because no one else needs this and tests would fail if they are comparing
// responses
resp.remove(resp.size() - 1);
Map<?, ?> invalidStates = (Map<?, ?>) o;
for (Map.Entry<?, ?> e : invalidStates.entrySet()) {
getDocCollection((String) e.getKey(), (Integer) e.getValue());
}
}
} catch (Exception exc) {
Throwable rootCause = SolrException.getRootCause(exc);
// don't do retry support for admin requests
// or if the request doesn't have a collection specified
// or request is v2 api and its method is not GET
if (inputCollections.isEmpty()
|| isAdmin
|| (request instanceof V2Request && request.getMethod() != SolrRequest.METHOD.GET)) {
if (exc instanceof SolrServerException) {
throw (SolrServerException) exc;
} else if (exc instanceof IOException) {
throw (IOException) exc;
} else if (exc instanceof RuntimeException) {
throw (RuntimeException) exc;
} else {
throw new SolrServerException(rootCause);
}
}
int errorCode =
(rootCause instanceof SolrException)
? ((SolrException) rootCause).code()
: SolrException.ErrorCode.UNKNOWN.code;
boolean wasCommError =
(rootCause instanceof ConnectException
|| rootCause instanceof SocketException
|| wasCommError(rootCause));
if (wasCommError
|| (exc instanceof RouteException
&& (errorCode == 503)) // 404 because the core does not exist 503 service unavailable
// TODO there are other reasons for 404. We need to change the solr response format from HTML
// to structured data to know that
) {
// it was a communication error. it is likely that
// the node to which the request to be sent is down . So , expire the state
// so that the next attempt would fetch the fresh state
// just re-read state for all of them, if it has not been retried
// in retryExpiryTime time
if (requestedCollections != null) {
for (DocCollection ext : requestedCollections) {
ExpiringCachedDocCollection cacheEntry = collectionStateCache.get(ext.getName());
if (cacheEntry == null) continue;
cacheEntry.maybeStale = true;
}
}
if (retryCount < MAX_STALE_RETRIES) { // if it is a communication error , we must try again
// may be, we have a stale version of the collection state
// and we could not get any information from the server
// it is probably not worth trying again and again because
// the state would not have been updated
log.info(
"Request to collection {} failed due to ({}) {}, retry={} maxRetries={} commError={} errorCode={} - retrying",
inputCollections,
errorCode,
rootCause,
retryCount,
MAX_STALE_RETRIES,
wasCommError,
errorCode);
return requestWithRetryOnStaleState(request, retryCount + 1, inputCollections);
}
} else {
log.info("request was not communication error it seems");
}
log.info(
"Request to collection {} failed due to ({}) {}, retry={} maxRetries={} commError={} errorCode={} ",
inputCollections,
errorCode,
rootCause,
retryCount,
MAX_STALE_RETRIES,
wasCommError,
errorCode);
boolean stateWasStale = false;
if (retryCount < MAX_STALE_RETRIES
&& requestedCollections != null
&& !requestedCollections.isEmpty()
&& (SolrException.ErrorCode.getErrorCode(errorCode)
== SolrException.ErrorCode.INVALID_STATE
|| errorCode == 404)) {
// cached state for one or more external collections was stale
// re-issue request using updated state
stateWasStale = true;
// just re-read state for all of them, which is a little heavy handed but hopefully a rare
// occurrence
for (DocCollection ext : requestedCollections) {
collectionStateCache.remove(ext.getName());
}
}