forked from apache/solr
-
Notifications
You must be signed in to change notification settings - Fork 0
/
CollectionsHandler.java
2109 lines (1973 loc) · 90.1 KB
/
CollectionsHandler.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.handler.admin;
import static org.apache.solr.client.solrj.response.RequestStatusState.COMPLETED;
import static org.apache.solr.client.solrj.response.RequestStatusState.FAILED;
import static org.apache.solr.client.solrj.response.RequestStatusState.NOT_FOUND;
import static org.apache.solr.client.solrj.response.RequestStatusState.RUNNING;
import static org.apache.solr.client.solrj.response.RequestStatusState.SUBMITTED;
import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET_EMPTY;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.CREATE_NODE_SET_SHUFFLE;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.NUM_SLICES;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ONLY_IF_DOWN;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.REQUESTID;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARDS_PROP;
import static org.apache.solr.cloud.api.collections.CollectionHandlingUtils.SHARD_UNIQUE;
import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionAdminParams.ALIAS;
import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
import static org.apache.solr.common.params.CollectionAdminParams.PER_REPLICA_STATE;
import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_NAME;
import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_VALUE;
import static org.apache.solr.common.params.CollectionAdminParams.SKIP_NODE_ASSIGNMENT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ALIASPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BACKUP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.COLLECTIONPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.COLSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATEALIAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESNAPSHOT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEALIAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEBACKUP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETENODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESNAPSHOT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DISTRIBUTEDAPIPROCESSING;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.FORCELEADER;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.LIST;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.LISTALIASES;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.LISTBACKUP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.LISTSNAPSHOTS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_COLL_TASK;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.OVERSEERSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REBALANCELEADERS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REINDEXCOLLECTION;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RENAME;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REPLACENODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REQUESTSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RESTORE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.SPLITSHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.SYNCSHARD;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonAdminParams.IN_PLACE_MOVE;
import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
import static org.apache.solr.common.params.CommonAdminParams.SPLIT_BY_PREFIX;
import static org.apache.solr.common.params.CommonAdminParams.SPLIT_FUZZ;
import static org.apache.solr.common.params.CommonAdminParams.SPLIT_METHOD;
import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.params.CommonParams.TIMING;
import static org.apache.solr.common.params.CommonParams.VALUE_LONG;
import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
import static org.apache.solr.common.params.CoreAdminParams.BACKUP_PURGE_UNUSED;
import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
import static org.apache.solr.common.params.CoreAdminParams.DATA_DIR;
import static org.apache.solr.common.params.CoreAdminParams.DELETE_DATA_DIR;
import static org.apache.solr.common.params.CoreAdminParams.DELETE_INDEX;
import static org.apache.solr.common.params.CoreAdminParams.DELETE_INSTANCE_DIR;
import static org.apache.solr.common.params.CoreAdminParams.INSTANCE_DIR;
import static org.apache.solr.common.params.CoreAdminParams.MAX_NUM_BACKUP_POINTS;
import static org.apache.solr.common.params.CoreAdminParams.ULOG_DIR;
import static org.apache.solr.common.params.ShardParams._ROUTE_;
import static org.apache.solr.common.util.StrUtils.formatString;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.api.AnnotatedApi;
import org.apache.solr.api.Api;
import org.apache.solr.api.JerseyResource;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
import org.apache.solr.cloud.OverseerSolrResponse;
import org.apache.solr.cloud.OverseerSolrResponseSerializer;
import org.apache.solr.cloud.OverseerTaskQueue;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.ZkController.NotInClusterStateException;
import org.apache.solr.cloud.ZkShardTerms;
import org.apache.solr.cloud.api.collections.DistributedCollectionConfigSetCommandRunner;
import org.apache.solr.cloud.api.collections.ReindexCollectionCmd;
import org.apache.solr.cloud.api.collections.RoutedAlias;
import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterProperties;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.CollectionProperties;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocCollection.CollectionStateProps;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Replica.State;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.RequiredSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.backup.BackupFilePaths;
import org.apache.solr.core.backup.BackupId;
import org.apache.solr.core.backup.BackupManager;
import org.apache.solr.core.backup.BackupProperties;
import org.apache.solr.core.backup.repository.BackupRepository;
import org.apache.solr.core.snapshots.CollectionSnapshotMetaData;
import org.apache.solr.core.snapshots.SolrSnapshotManager;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.admin.api.*;
import org.apache.solr.handler.api.V2ApiUtils;
import org.apache.solr.jersey.SolrJerseyResponse;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.apache.solr.util.tracing.TraceUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class CollectionsHandler extends RequestHandlerBase implements PermissionNameProvider {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final CoreContainer coreContainer;
private final Optional<DistributedCollectionConfigSetCommandRunner>
distributedCollectionConfigSetCommandRunner;
public CollectionsHandler() {
// Unlike most request handlers, CoreContainer initialization
// should happen in the constructor...
this(null);
}
/**
* Overloaded ctor to inject CoreContainer into the handler.
*
* @param coreContainer Core Container of the solr webapp installed.
*/
public CollectionsHandler(final CoreContainer coreContainer) {
this.coreContainer = coreContainer;
distributedCollectionConfigSetCommandRunner =
coreContainer != null
? coreContainer.getDistributedCollectionCommandRunner()
: Optional.empty();
}
@Override
public PermissionNameProvider.Name getPermissionName(AuthorizationContext ctx) {
String action = ctx.getParams().get("action");
if (action == null) return PermissionNameProvider.Name.COLL_READ_PERM;
CollectionParams.CollectionAction collectionAction =
CollectionParams.CollectionAction.get(action);
if (collectionAction == null) return null;
return collectionAction.isWrite
? PermissionNameProvider.Name.COLL_EDIT_PERM
: PermissionNameProvider.Name.COLL_READ_PERM;
}
@Override
public final void init(NamedList<?> args) {}
/**
* The instance of CoreContainer this handler handles. This should be the CoreContainer instance
* that created this handler.
*
* @return a CoreContainer instance
*/
public CoreContainer getCoreContainer() {
return this.coreContainer;
}
protected void copyFromClusterProp(Map<String, Object> props, String prop) throws IOException {
if (props.get(prop) != null) return; // if it's already specified , return
Object defVal =
new ClusterProperties(coreContainer.getZkController().getZkStateReader().getZkClient())
.getClusterProperty(
List.of(CollectionAdminParams.DEFAULTS, CollectionAdminParams.COLLECTION, prop),
null);
if (defVal != null) props.put(prop, String.valueOf(defVal));
}
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
// Make sure the cores is enabled
CoreContainer cores = checkErrors();
// Pick the action
SolrParams params = req.getParams();
String a = params.get(CoreAdminParams.ACTION);
if (a == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param");
}
CollectionAction action = CollectionAction.get(a);
if (action == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
}
// Initial logging/tracing setup
final String collection = params.get(COLLECTION);
MDCLoggingContext.setCollection(collection);
TraceUtils.setDbInstance(req, collection);
if (log.isDebugEnabled()) {
log.debug(
"Invoked Collection Action: {} with params {}", action.toLower(), req.getParamString());
}
switch (action) {
case ADDREPLICAPROP:
// Convert query-params to v2 acceptable format
final RequiredSolrParams requiredParams = req.getParams().required();
final AddReplicaPropertyAPI.AddReplicaPropertyRequestBody requestBody =
new AddReplicaPropertyAPI.AddReplicaPropertyRequestBody();
requestBody.value = requiredParams.get(PROPERTY_VALUE_PROP);
requestBody.shardUnique = req.getParams().getBool(SHARD_UNIQUE);
final String propName = requiredParams.get(PROPERTY_PROP);
final String trimmedPropName =
propName.startsWith(PROPERTY_PREFIX)
? propName.substring(PROPERTY_PREFIX.length())
: propName;
final AddReplicaPropertyAPI addReplicaPropertyAPI =
new AddReplicaPropertyAPI(coreContainer, req, rsp);
final SolrJerseyResponse addReplicaPropResponse =
addReplicaPropertyAPI.addReplicaProperty(
requiredParams.get(COLLECTION_PROP),
requiredParams.get(SHARD_ID_PROP),
requiredParams.get(REPLICA_PROP),
trimmedPropName,
requestBody);
V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, addReplicaPropResponse);
break;
default:
CollectionOperation operation = CollectionOperation.get(action);
invokeAction(req, rsp, cores, action, operation);
rsp.setHttpCaching(false);
break;
}
}
protected CoreContainer checkErrors() {
CoreContainer cores = getCoreContainer();
AdminAPIBase.validateZooKeeperAwareCoreContainer(cores);
return cores;
}
@SuppressWarnings({"unchecked"})
void invokeAction(
SolrQueryRequest req,
SolrQueryResponse rsp,
CoreContainer cores,
CollectionAction action,
CollectionOperation operation)
throws Exception {
if (!coreContainer.isZooKeeperAware()) {
throw new SolrException(
BAD_REQUEST, "Invalid request. collections can be accessed only in SolrCloud mode");
}
Map<String, Object> props = operation.execute(req, rsp, this);
if (props == null) {
return;
}
String asyncId = req.getParams().get(ASYNC);
if (asyncId != null) {
props.put(ASYNC, asyncId);
}
props.put(QUEUE_OPERATION, operation.action.toLower());
ZkNodeProps zkProps = new ZkNodeProps(props);
final SolrResponse overseerResponse;
overseerResponse = submitCollectionApiCommand(zkProps, operation.action, operation.timeOut);
rsp.getValues().addAll(overseerResponse.getResponse());
Exception exp = overseerResponse.getException();
if (exp != null) {
rsp.setException(exp);
}
// Even if Overseer does wait for the collection to be created, it sees a different cluster
// state than this node, so this wait is required to make sure the local node Zookeeper watches
// fired and now see the collection.
if (action.equals(CollectionAction.CREATE) && asyncId == null) {
if (rsp.getException() == null) {
waitForActiveCollection(zkProps.getStr(NAME), cores, overseerResponse);
}
}
}
static final Set<String> KNOWN_ROLES = Set.of("overseer");
public static long DEFAULT_COLLECTION_OP_TIMEOUT = 180 * 1000;
public SolrResponse submitCollectionApiCommand(ZkNodeProps m, CollectionAction action)
throws KeeperException, InterruptedException {
return submitCollectionApiCommand(m, action, DEFAULT_COLLECTION_OP_TIMEOUT);
}
public static SolrResponse submitCollectionApiCommand(
CoreContainer coreContainer,
Optional<DistributedCollectionConfigSetCommandRunner>
distributedCollectionConfigSetCommandRunner,
ZkNodeProps m,
CollectionAction action,
long timeout)
throws KeeperException, InterruptedException {
// Collection API messages are either sent to Overseer and processed there, or processed
// locally. Distributing Collection API implies we're also distributing Cluster State Updates.
// Indeed collection creation with non distributed cluster state updates requires for "Per
// Replica States" that the Collection API be running on Overseer, which means that it is not
// possible to distributed Collection API while keeping cluster state updates on Overseer. See
// the call to CollectionCommandContext.submitIntraProcessMessage() in
// CreateCollectionCmd.call() which can only be done if the Collection API command runs on the
// same JVM as the Overseer based cluster state update... The configuration handling includes
// these checks to not allow distributing collection API without distributing cluster state
// updates (but the other way around is ok). See constructor of CloudConfig.
if (distributedCollectionConfigSetCommandRunner.isPresent()) {
return distributedCollectionConfigSetCommandRunner
.get()
.runCollectionCommand(m, action, timeout);
} else { // Sending the Collection API message to Overseer via a Zookeeper queue
String operation = m.getStr(QUEUE_OPERATION);
if (operation == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, "missing key " + QUEUE_OPERATION);
}
if (m.get(ASYNC) != null) {
String asyncId = m.getStr(ASYNC);
NamedList<Object> r = new NamedList<>();
if (coreContainer.getZkController().claimAsyncId(asyncId)) {
boolean success = false;
try {
coreContainer.getZkController().getOverseerCollectionQueue().offer(Utils.toJSON(m));
success = true;
} finally {
if (!success) {
try {
coreContainer.getZkController().clearAsyncId(asyncId);
} catch (Exception e) {
// let the original exception bubble up
log.error("Unable to release async ID={}", asyncId, e);
SolrZkClient.checkInterrupted(e);
}
}
}
} else {
r.add("error", "Task with the same requestid already exists. (" + asyncId + ")");
}
r.add(CoreAdminParams.REQUESTID, m.get(ASYNC));
return new OverseerSolrResponse(r);
}
long time = System.nanoTime();
QueueEvent event =
coreContainer
.getZkController()
.getOverseerCollectionQueue()
.offer(Utils.toJSON(m), timeout);
if (event.getBytes() != null) {
return OverseerSolrResponseSerializer.deserialize(event.getBytes());
} else {
if (System.nanoTime() - time
>= TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)) {
throw new SolrException(
ErrorCode.SERVER_ERROR,
operation + " the collection time out:" + timeout / 1000 + "s");
} else if (event.getWatchedEvent() != null) {
throw new SolrException(
ErrorCode.SERVER_ERROR,
operation
+ " the collection error [Watcher fired on path: "
+ event.getWatchedEvent().getPath()
+ " state: "
+ event.getWatchedEvent().getState()
+ " type "
+ event.getWatchedEvent().getType()
+ "]");
} else {
throw new SolrException(
ErrorCode.SERVER_ERROR, operation + " the collection unknown case");
}
}
}
}
public SolrResponse submitCollectionApiCommand(
ZkNodeProps m, CollectionAction action, long timeout)
throws KeeperException, InterruptedException {
return submitCollectionApiCommand(
coreContainer, distributedCollectionConfigSetCommandRunner, m, action, timeout);
}
private boolean overseerCollectionQueueContains(String asyncId)
throws KeeperException, InterruptedException {
OverseerTaskQueue collectionQueue =
coreContainer.getZkController().getOverseerCollectionQueue();
return collectionQueue.containsTaskWithRequestId(ASYNC, asyncId);
}
/**
* Copy prefixed params into a map. There must only be one value for these parameters.
*
* @param params The source of params from which copies should be made
* @param props The map into which param names and values should be copied as keys and values
* respectively
* @param prefix The prefix to select.
* @return the map supplied in the props parameter, modified to contain the prefixed params.
*/
private static Map<String, Object> copyPropertiesWithPrefix(
SolrParams params, Map<String, Object> props, String prefix) {
Iterator<String> iter = params.getParameterNamesIterator();
while (iter.hasNext()) {
String param = iter.next();
if (param.startsWith(prefix)) {
final String[] values = params.getParams(param);
if (values.length != 1) {
throw new SolrException(
BAD_REQUEST, "Only one value can be present for parameter " + param);
}
props.put(param, values[0]);
}
}
return props;
}
public static ModifiableSolrParams params(String... params) {
ModifiableSolrParams msp = new ModifiableSolrParams();
for (int i = 0; i < params.length; i += 2) {
msp.add(params[i], params[i + 1]);
}
return msp;
}
//////////////////////// SolrInfoMBeans methods //////////////////////
@Override
public String getDescription() {
return "Manage SolrCloud Collections";
}
@Override
public Category getCategory() {
return Category.ADMIN;
}
private static void createSysConfigSet(CoreContainer coreContainer)
throws KeeperException, InterruptedException {
SolrZkClient zk = coreContainer.getZkController().getZkStateReader().getZkClient();
ZkCmdExecutor cmdExecutor = new ZkCmdExecutor(zk.getZkClientTimeout());
cmdExecutor.ensureExists(ZkStateReader.CONFIGS_ZKNODE, zk);
cmdExecutor.ensureExists(
ZkStateReader.CONFIGS_ZKNODE + "/" + CollectionAdminParams.SYSTEM_COLL, zk);
try {
String path =
ZkStateReader.CONFIGS_ZKNODE + "/" + CollectionAdminParams.SYSTEM_COLL + "/schema.xml";
byte[] data =
IOUtils.toByteArray(
CollectionsHandler.class.getResourceAsStream("/SystemCollectionSchema.xml"));
assert data != null && data.length > 0;
cmdExecutor.ensureExists(path, data, CreateMode.PERSISTENT, zk);
path =
ZkStateReader.CONFIGS_ZKNODE
+ "/"
+ CollectionAdminParams.SYSTEM_COLL
+ "/solrconfig.xml";
data =
IOUtils.toByteArray(
CollectionsHandler.class.getResourceAsStream("/SystemCollectionSolrConfig.xml"));
assert data != null && data.length > 0;
cmdExecutor.ensureExists(path, data, CreateMode.PERSISTENT, zk);
} catch (IOException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, e);
}
}
private static void addStatusToResponse(
NamedList<Object> results, RequestStatusState state, String msg) {
SimpleOrderedMap<String> status = new SimpleOrderedMap<>();
status.add("state", state.getKey());
status.add("msg", msg);
results.add("status", status);
}
public enum CollectionOperation implements CollectionOp {
CREATE_OP(
CREATE,
(req, rsp, h) -> {
Map<String, Object> props = copy(req.getParams().required(), null, NAME);
props.put("fromApi", "true");
copy(
req.getParams(),
props,
REPLICATION_FACTOR,
COLL_CONF,
NUM_SLICES,
CREATE_NODE_SET,
CREATE_NODE_SET_SHUFFLE,
SHARDS_PROP,
PULL_REPLICAS,
TLOG_REPLICAS,
NRT_REPLICAS,
WAIT_FOR_FINAL_STATE,
PER_REPLICA_STATE,
ALIAS);
if (props.get(REPLICATION_FACTOR) != null && props.get(NRT_REPLICAS) != null) {
// TODO: Remove this in 8.0 . Keep this for SolrJ client back-compat. See SOLR-11676 for
// more details
int replicationFactor = Integer.parseInt((String) props.get(REPLICATION_FACTOR));
int nrtReplicas = Integer.parseInt((String) props.get(NRT_REPLICAS));
if (replicationFactor != nrtReplicas) {
throw new SolrException(
ErrorCode.BAD_REQUEST,
"Cannot specify both replicationFactor and nrtReplicas as they mean the same thing");
}
}
if (props.get(REPLICATION_FACTOR) != null) {
props.put(NRT_REPLICAS, props.get(REPLICATION_FACTOR));
} else if (props.get(NRT_REPLICAS) != null) {
props.put(REPLICATION_FACTOR, props.get(NRT_REPLICAS));
}
final String collectionName =
SolrIdentifierValidator.validateCollectionName((String) props.get(NAME));
final String shardsParam = (String) props.get(SHARDS_PROP);
if (StringUtils.isNotEmpty(shardsParam)) {
verifyShardsParam(shardsParam);
}
if (CollectionAdminParams.SYSTEM_COLL.equals(collectionName)) {
// We must always create a .system collection with only a single shard
props.put(NUM_SLICES, 1);
props.remove(SHARDS_PROP);
createSysConfigSet(h.coreContainer);
}
if (shardsParam == null) h.copyFromClusterProp(props, NUM_SLICES);
for (String prop : Set.of(NRT_REPLICAS, PULL_REPLICAS, TLOG_REPLICAS))
h.copyFromClusterProp(props, prop);
copyPropertiesWithPrefix(req.getParams(), props, PROPERTY_PREFIX);
return copyPropertiesWithPrefix(req.getParams(), props, "router.");
}),
@SuppressWarnings({"unchecked"})
COLSTATUS_OP(
COLSTATUS,
(req, rsp, h) -> {
Map<String, Object> props =
copy(
req.getParams(),
null,
COLLECTION_PROP,
ColStatus.CORE_INFO_PROP,
ColStatus.SEGMENTS_PROP,
ColStatus.FIELD_INFO_PROP,
ColStatus.RAW_SIZE_PROP,
ColStatus.RAW_SIZE_SUMMARY_PROP,
ColStatus.RAW_SIZE_DETAILS_PROP,
ColStatus.RAW_SIZE_SAMPLING_PERCENT_PROP,
ColStatus.SIZE_INFO_PROP);
new ColStatus(
h.coreContainer.getSolrClientCache(),
h.coreContainer.getZkController().getZkStateReader().getClusterState(),
new ZkNodeProps(props))
.getColStatus(rsp.getValues());
return null;
}),
DELETE_OP(
DELETE,
(req, rsp, h) -> {
Map<String, Object> map = copy(req.getParams().required(), null, NAME);
return copy(req.getParams(), map, FOLLOW_ALIASES);
}),
// XXX should this command support followAliases?
RELOAD_OP(
RELOAD,
(req, rsp, h) -> {
Map<String, Object> map = copy(req.getParams().required(), null, NAME);
return copy(req.getParams(), map);
}),
RENAME_OP(
RENAME,
(req, rsp, h) -> {
Map<String, Object> map =
copy(req.getParams().required(), null, NAME, CollectionAdminParams.TARGET);
return copy(req.getParams(), map, FOLLOW_ALIASES);
}),
REINDEXCOLLECTION_OP(
REINDEXCOLLECTION,
(req, rsp, h) -> {
Map<String, Object> m = copy(req.getParams().required(), null, NAME);
copy(
req.getParams(),
m,
ReindexCollectionCmd.COMMAND,
ReindexCollectionCmd.REMOVE_SOURCE,
ReindexCollectionCmd.TARGET,
ZkStateReader.CONFIGNAME_PROP,
NUM_SLICES,
NRT_REPLICAS,
PULL_REPLICAS,
TLOG_REPLICAS,
REPLICATION_FACTOR,
CREATE_NODE_SET,
CREATE_NODE_SET_SHUFFLE,
"shards",
CommonParams.ROWS,
CommonParams.Q,
CommonParams.FL,
FOLLOW_ALIASES);
if (req.getParams().get("collection." + ZkStateReader.CONFIGNAME_PROP) != null) {
m.put(
ZkStateReader.CONFIGNAME_PROP,
req.getParams().get("collection." + ZkStateReader.CONFIGNAME_PROP));
}
copyPropertiesWithPrefix(req.getParams(), m, "router.");
return m;
}),
SYNCSHARD_OP(
SYNCSHARD,
(req, rsp, h) -> {
String extCollection = req.getParams().required().get("collection");
String collection = h.coreContainer.getAliases().resolveSimpleAlias(extCollection);
String shard = req.getParams().required().get("shard");
ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
DocCollection docCollection = clusterState.getCollection(collection);
ZkNodeProps leaderProps = docCollection.getLeader(shard);
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
try (HttpSolrClient client =
new Builder(nodeProps.getBaseUrl())
.withConnectionTimeout(15000)
.withSocketTimeout(60000)
.build()) {
RequestSyncShard reqSyncShard = new RequestSyncShard();
reqSyncShard.setCollection(collection);
reqSyncShard.setShard(shard);
reqSyncShard.setCoreName(nodeProps.getCoreName());
client.request(reqSyncShard);
}
return null;
}),
CREATEALIAS_OP(
CREATEALIAS,
(req, rsp, h) -> {
String alias = req.getParams().get(NAME);
SolrIdentifierValidator.validateAliasName(alias);
String collections = req.getParams().get("collections");
RoutedAlias routedAlias = null;
Exception ex = null;
HashMap<String, Object> possiblyModifiedParams = new HashMap<>();
try {
// note that RA specific validation occurs here.
req.getParams().toMap(possiblyModifiedParams);
@SuppressWarnings({"unchecked", "rawtypes"})
// This is awful because RoutedAlias lies about what types it wants
Map<String, String> temp = (Map<String, String>) (Map) possiblyModifiedParams;
routedAlias = RoutedAlias.fromProps(alias, temp);
} catch (SolrException e) {
// we'll throw this later if we are in fact creating a routed alias.
ex = e;
}
ModifiableSolrParams finalParams = new ModifiableSolrParams();
for (Map.Entry<String, Object> entry : possiblyModifiedParams.entrySet()) {
if (entry.getValue().getClass().isArray()) {
// v2 api hits this case
for (Object o : (Object[]) entry.getValue()) {
finalParams.add(entry.getKey(), o.toString());
}
} else {
finalParams.add(entry.getKey(), entry.getValue().toString());
}
}
if (collections != null) {
if (routedAlias != null) {
throw new SolrException(
BAD_REQUEST, "Collections cannot be specified when creating a routed alias.");
} else {
//////////////////////////////////////
// Regular alias creation indicated //
//////////////////////////////////////
return copy(finalParams.required(), null, NAME, "collections");
}
} else {
if (routedAlias != null) {
CoreContainer coreContainer1 = h.getCoreContainer();
Aliases aliases = coreContainer1.getAliases();
String aliasName = routedAlias.getAliasName();
if (aliases.hasAlias(aliasName) && !aliases.isRoutedAlias(aliasName)) {
throw new SolrException(
BAD_REQUEST,
"Cannot add routing parameters to existing non-routed Alias: " + aliasName);
}
}
}
/////////////////////////////////////////////////
// We are creating a routed alias from here on //
/////////////////////////////////////////////////
// If our prior creation attempt had issues expose them now.
if (ex != null) {
throw ex;
}
// Now filter out just the parameters we care about from the request
assert routedAlias != null;
Map<String, Object> result = copy(finalParams, null, routedAlias.getRequiredParams());
copy(finalParams, result, routedAlias.getOptionalParams());
ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // without prefix
// add to result params that start with "create-collection.".
// Additionally, save these without the prefix to createCollParams
for (Map.Entry<String, String[]> entry : finalParams) {
final String p = entry.getKey();
if (p.startsWith(CREATE_COLLECTION_PREFIX)) {
// This is what SolrParams#getAll(Map, Collection)} does
final String[] v = entry.getValue();
if (v.length == 1) {
result.put(p, v[0]);
} else {
result.put(p, v);
}
createCollParams.set(p.substring(CREATE_COLLECTION_PREFIX.length()), v);
}
}
// Verify that the create-collection prefix'ed params appear to be valid.
if (createCollParams.get(NAME) != null) {
throw new SolrException(
BAD_REQUEST,
"routed aliases calculate names for their "
+ "dependent collections, you cannot specify the name.");
}
if (createCollParams.get(COLL_CONF) == null) {
throw new SolrException(
SolrException.ErrorCode.BAD_REQUEST, "We require an explicit " + COLL_CONF);
}
// note: could insist on a config name here as well.... or wait to throw at overseer
createCollParams.add(NAME, "TMP_name_TMP_name_TMP"); // just to pass validation
CREATE_OP.execute(
new LocalSolrQueryRequest(null, createCollParams), rsp, h); // ignore results
return result;
}),
DELETEALIAS_OP(DELETEALIAS, (req, rsp, h) -> copy(req.getParams().required(), null, NAME)),
/**
* Change properties for an alias (use CREATEALIAS_OP to change the actual value of the alias)
*/
ALIASPROP_OP(
ALIASPROP,
(req, rsp, h) -> {
Map<String, Object> params = copy(req.getParams().required(), null, NAME);
// Note: success/no-op in the event of no properties supplied is intentional. Keeps code
// simple and one less case for api-callers to check for.
return convertPrefixToMap(req.getParams(), params, "property");
}),
/** List the aliases and associated properties. */
@SuppressWarnings({"unchecked"})
LISTALIASES_OP(
LISTALIASES,
(req, rsp, h) -> {
ZkStateReader zkStateReader = h.coreContainer.getZkController().getZkStateReader();
// if someone calls listAliases, lets ensure we return an up to date response
zkStateReader.aliasesManager.update();
Aliases aliases = zkStateReader.getAliases();
if (aliases != null) {
// the aliases themselves...
rsp.getValues().add("aliases", aliases.getCollectionAliasMap());
// Any properties for the above aliases.
Map<String, Map<String, String>> meta = new LinkedHashMap<>();
for (String alias : aliases.getCollectionAliasListMap().keySet()) {
Map<String, String> collectionAliasProperties =
aliases.getCollectionAliasProperties(alias);
if (!collectionAliasProperties.isEmpty()) {
meta.put(alias, collectionAliasProperties);
}
}
rsp.getValues().add("properties", meta);
}
return null;
}),
SPLITSHARD_OP(
SPLITSHARD,
DEFAULT_COLLECTION_OP_TIMEOUT * 5,
(req, rsp, h) -> {
String name = req.getParams().required().get(COLLECTION_PROP);
// TODO : add support for multiple shards
String shard = req.getParams().get(SHARD_ID_PROP);
String rangesStr = req.getParams().get(CoreAdminParams.RANGES);
String splitKey = req.getParams().get("split.key");
String numSubShards = req.getParams().get(NUM_SUB_SHARDS);
String fuzz = req.getParams().get(SPLIT_FUZZ);
if (splitKey == null && shard == null) {
throw new SolrException(
ErrorCode.BAD_REQUEST, "At least one of shard, or split.key should be specified.");
}
if (splitKey != null && shard != null) {
throw new SolrException(
ErrorCode.BAD_REQUEST, "Only one of 'shard' or 'split.key' should be specified");
}
if (splitKey != null && rangesStr != null) {
throw new SolrException(
ErrorCode.BAD_REQUEST, "Only one of 'ranges' or 'split.key' should be specified");
}
if (numSubShards != null && (splitKey != null || rangesStr != null)) {
throw new SolrException(
ErrorCode.BAD_REQUEST,
"numSubShards can not be specified with split.key or ranges parameters");
}
if (fuzz != null && (splitKey != null || rangesStr != null)) {
throw new SolrException(
ErrorCode.BAD_REQUEST,
"fuzz can not be specified with split.key or ranges parameters");
}
Map<String, Object> map =
copy(
req.getParams(),
null,
COLLECTION_PROP,
SHARD_ID_PROP,
"split.key",
CoreAdminParams.RANGES,
WAIT_FOR_FINAL_STATE,
TIMING,
SPLIT_METHOD,
NUM_SUB_SHARDS,
SPLIT_FUZZ,
SPLIT_BY_PREFIX,
FOLLOW_ALIASES);
return copyPropertiesWithPrefix(req.getParams(), map, PROPERTY_PREFIX);
}),
DELETESHARD_OP(
DELETESHARD,
(req, rsp, h) -> {
Map<String, Object> map =
copy(req.getParams().required(), null, COLLECTION_PROP, SHARD_ID_PROP);
copy(
req.getParams(),
map,
DELETE_INDEX,
DELETE_DATA_DIR,
DELETE_INSTANCE_DIR,
FOLLOW_ALIASES);
return map;
}),
FORCELEADER_OP(
FORCELEADER,
(req, rsp, h) -> {
forceLeaderElection(req, h);
return null;
}),
CREATESHARD_OP(
CREATESHARD,
(req, rsp, h) -> {
Map<String, Object> map =
copy(req.getParams().required(), null, COLLECTION_PROP, SHARD_ID_PROP);
ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
final String newShardName =
SolrIdentifierValidator.validateShardName(req.getParams().get(SHARD_ID_PROP));
boolean followAliases = req.getParams().getBool(FOLLOW_ALIASES, false);
String extCollectionName = req.getParams().get(COLLECTION_PROP);
String collectionName =
followAliases
? h.coreContainer
.getZkController()
.getZkStateReader()
.getAliases()
.resolveSimpleAlias(extCollectionName)
: extCollectionName;
if (!ImplicitDocRouter.NAME.equals(
((Map<?, ?>)
clusterState
.getCollection(collectionName)
.get(CollectionStateProps.DOC_ROUTER))
.get(NAME)))
throw new SolrException(
ErrorCode.BAD_REQUEST, "shards can be added only to 'implicit' collections");
copy(
req.getParams(),
map,
REPLICATION_FACTOR,
NRT_REPLICAS,
TLOG_REPLICAS,
PULL_REPLICAS,
CREATE_NODE_SET,
WAIT_FOR_FINAL_STATE,
FOLLOW_ALIASES);