-
Notifications
You must be signed in to change notification settings - Fork 1.8k
/
SegmentReplicationIT.java
1837 lines (1627 loc) · 83.2 KB
/
SegmentReplicationIT.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
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.indices.replication;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.StandardDirectoryReader;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
import org.opensearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.opensearch.action.admin.indices.alias.Alias;
import org.opensearch.action.admin.indices.flush.FlushRequest;
import org.opensearch.action.admin.indices.stats.IndicesStatsRequest;
import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
import org.opensearch.action.get.GetResponse;
import org.opensearch.action.get.MultiGetRequest;
import org.opensearch.action.get.MultiGetResponse;
import org.opensearch.action.index.IndexResponse;
import org.opensearch.action.search.CreatePitAction;
import org.opensearch.action.search.CreatePitRequest;
import org.opensearch.action.search.CreatePitResponse;
import org.opensearch.action.search.DeletePitAction;
import org.opensearch.action.search.DeletePitRequest;
import org.opensearch.action.search.PitTestsUtil;
import org.opensearch.action.search.SearchResponse;
import org.opensearch.action.search.SearchType;
import org.opensearch.action.support.WriteRequest;
import org.opensearch.action.termvectors.TermVectorsRequestBuilder;
import org.opensearch.action.termvectors.TermVectorsResponse;
import org.opensearch.action.update.UpdateResponse;
import org.opensearch.client.Requests;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.health.ClusterHealthStatus;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.routing.Preference;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.cluster.routing.ShardRoutingState;
import org.opensearch.cluster.routing.allocation.command.CancelAllocationCommand;
import org.opensearch.common.action.ActionFuture;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.lucene.index.OpenSearchDirectoryReader;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
import org.opensearch.core.index.shard.ShardId;
import org.opensearch.core.xcontent.XContentBuilder;
import org.opensearch.index.IndexModule;
import org.opensearch.index.ReplicationStats;
import org.opensearch.index.SegmentReplicationPerGroupStats;
import org.opensearch.index.SegmentReplicationPressureService;
import org.opensearch.index.SegmentReplicationShardStats;
import org.opensearch.index.codec.CodecService;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.EngineConfig;
import org.opensearch.index.engine.NRTReplicationReaderManager;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.indices.recovery.FileChunkRequest;
import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint;
import org.opensearch.indices.replication.common.ReplicationType;
import org.opensearch.node.NodeClosedException;
import org.opensearch.search.SearchService;
import org.opensearch.search.builder.PointInTimeBuilder;
import org.opensearch.search.internal.PitReaderContext;
import org.opensearch.search.sort.SortOrder;
import org.opensearch.test.BackgroundIndexer;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.junit.annotations.TestLogging;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.transport.TransportService;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static java.util.Arrays.asList;
import static org.opensearch.action.search.PitTestsUtil.assertSegments;
import static org.opensearch.action.search.SearchContextId.decode;
import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.opensearch.index.query.QueryBuilders.boolQuery;
import static org.opensearch.index.query.QueryBuilders.matchAllQuery;
import static org.opensearch.index.query.QueryBuilders.matchQuery;
import static org.opensearch.index.query.QueryBuilders.rangeQuery;
import static org.opensearch.index.query.QueryBuilders.termQuery;
import static org.opensearch.indices.replication.SegmentReplicationTarget.REPLICATION_PREFIX;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAllSuccessful;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertSearchHits;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class SegmentReplicationIT extends SegmentReplicationBaseIT {
@Before
private void setup() {
internalCluster().startClusterManagerOnlyNode();
}
private static String indexOrAlias() {
return randomBoolean() ? INDEX_NAME : "alias";
}
public void testPrimaryStopped_ReplicaPromoted() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
refresh(INDEX_NAME);
waitForSearchableDocs(1, primary, replica);
// index another doc but don't refresh, we will ensure this is searchable once replica is promoted.
client().prepareIndex(INDEX_NAME).setId("2").setSource("bar", "baz").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
// stop the primary node - we only have one shard on here.
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary));
ensureYellowAndNoInitializingShards(INDEX_NAME);
final ShardRouting replicaShardRouting = getShardRoutingForNodeName(replica);
assertNotNull(replicaShardRouting);
assertTrue(replicaShardRouting + " should be promoted as a primary", replicaShardRouting.primary());
final SearchResponse response = client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get();
// new primary should have at least the doc count from the first set of segments.
assertTrue(response.getHits().getTotalHits().value >= 1);
// assert we can index into the new primary.
client().prepareIndex(INDEX_NAME).setId("3").setSource("bar", "baz").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
assertHitCount(client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 3);
// start another node, index another doc and replicate.
String nodeC = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
client().prepareIndex(INDEX_NAME).setId("4").setSource("baz", "baz").get();
refresh(INDEX_NAME);
waitForSearchableDocs(4, nodeC, replica);
verifyStoreContent();
}
public void testRestartPrimary() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
assertEquals(getNodeContainingPrimaryShard().getName(), primary);
final int initialDocCount = 1;
client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, replica, primary);
internalCluster().restartNode(primary);
ensureGreen(INDEX_NAME);
assertEquals(getNodeContainingPrimaryShard().getName(), replica);
flushAndRefresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, replica, primary);
verifyStoreContent();
}
public void testCancelPrimaryAllocation() throws Exception {
// this test cancels allocation on the primary - promoting the new replica and recreating the former primary as a replica.
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
final int initialDocCount = 1;
client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, replica, primary);
final IndexShard indexShard = getIndexShard(primary, INDEX_NAME);
client().admin()
.cluster()
.prepareReroute()
.add(new CancelAllocationCommand(INDEX_NAME, indexShard.shardId().id(), primary, true))
.execute()
.actionGet();
ensureGreen(INDEX_NAME);
assertEquals(getNodeContainingPrimaryShard().getName(), replica);
flushAndRefresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, replica, primary);
verifyStoreContent();
}
public void testReplicationAfterPrimaryRefreshAndFlush() throws Exception {
final String nodeA = internalCluster().startDataOnlyNode();
final String nodeB = internalCluster().startDataOnlyNode();
final Settings settings = Settings.builder()
.put(indexSettings())
.put(EngineConfig.INDEX_CODEC_SETTING.getKey(), randomFrom(new ArrayList<>(CODECS) {
{
add(CodecService.LUCENE_DEFAULT_CODEC);
}
}))
.build();
createIndex(INDEX_NAME, settings);
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(0, 10);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, nodeA, nodeB);
final int additionalDocCount = scaledRandomIntBetween(0, 10);
final int expectedHitCount = initialDocCount + additionalDocCount;
indexer.start(additionalDocCount);
waitForDocs(expectedHitCount, indexer);
flushAndRefresh(INDEX_NAME);
waitForSearchableDocs(expectedHitCount, nodeA, nodeB);
ensureGreen(INDEX_NAME);
verifyStoreContent();
}
}
public void testIndexReopenClose() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
final String replica = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(1, 10);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
flush(INDEX_NAME);
waitForSearchableDocs(initialDocCount, primary, replica);
}
logger.info("--> Closing the index ");
client().admin().indices().prepareClose(INDEX_NAME).get();
logger.info("--> Opening the index");
client().admin().indices().prepareOpen(INDEX_NAME).get();
ensureGreen(INDEX_NAME);
waitForSearchableDocs(initialDocCount, primary, replica);
verifyStoreContent();
}
public void testScrollWithConcurrentIndexAndSearch() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
final String replica = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureGreen(INDEX_NAME);
final List<ActionFuture<IndexResponse>> pendingIndexResponses = new ArrayList<>();
final List<ActionFuture<SearchResponse>> pendingSearchResponse = new ArrayList<>();
final int searchCount = randomIntBetween(1, 2);
final WriteRequest.RefreshPolicy refreshPolicy = randomFrom(WriteRequest.RefreshPolicy.values());
for (int i = 0; i < searchCount; i++) {
pendingIndexResponses.add(
client().prepareIndex(INDEX_NAME)
.setId(Integer.toString(i))
.setRefreshPolicy(refreshPolicy)
.setSource("field", "value" + i)
.execute()
);
flush(INDEX_NAME);
forceMerge();
}
final SearchResponse searchResponse = client().prepareSearch()
.setQuery(matchAllQuery())
.setIndices(INDEX_NAME)
.setRequestCache(false)
.setScroll(TimeValue.timeValueDays(1))
.setSize(10)
.get();
for (int i = searchCount; i < searchCount * 2; i++) {
pendingIndexResponses.add(
client().prepareIndex(INDEX_NAME)
.setId(Integer.toString(i))
.setRefreshPolicy(refreshPolicy)
.setSource("field", "value" + i)
.execute()
);
}
flush(INDEX_NAME);
forceMerge();
client().prepareClearScroll().addScrollId(searchResponse.getScrollId()).get();
assertBusy(() -> {
client().admin().indices().prepareRefresh().execute().actionGet();
assertTrue(pendingIndexResponses.stream().allMatch(ActionFuture::isDone));
assertTrue(pendingSearchResponse.stream().allMatch(ActionFuture::isDone));
}, 1, TimeUnit.MINUTES);
verifyStoreContent();
waitForSearchableDocs(INDEX_NAME, 2 * searchCount, List.of(primary, replica));
}
@TestLogging(reason = "Getting trace logs from replication package", value = "org.opensearch.indices.replication:TRACE")
public void testMultipleShards() throws Exception {
Settings indexSettings = Settings.builder()
.put(super.indexSettings())
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 3)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
.put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
.build();
final String nodeA = internalCluster().startDataOnlyNode();
final String nodeB = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME, indexSettings);
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(1, 10);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, nodeA, nodeB);
final int additionalDocCount = scaledRandomIntBetween(0, 10);
final int expectedHitCount = initialDocCount + additionalDocCount;
indexer.start(additionalDocCount);
waitForDocs(expectedHitCount, indexer);
flushAndRefresh(INDEX_NAME);
waitForSearchableDocs(expectedHitCount, nodeA, nodeB);
ensureGreen(INDEX_NAME);
verifyStoreContent();
}
}
public void testReplicationAfterForceMerge() throws Exception {
final String nodeA = internalCluster().startDataOnlyNode();
final String nodeB = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(0, 10);
final int additionalDocCount = scaledRandomIntBetween(0, 10);
final int expectedHitCount = initialDocCount + additionalDocCount;
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
flush(INDEX_NAME);
waitForSearchableDocs(initialDocCount, nodeA, nodeB);
// Index a second set of docs so we can merge into one segment.
indexer.start(additionalDocCount);
waitForDocs(expectedHitCount, indexer);
waitForSearchableDocs(expectedHitCount, nodeA, nodeB);
// Force a merge here so that the in memory SegmentInfos does not reference old segments on disk.
client().admin().indices().prepareForceMerge(INDEX_NAME).setMaxNumSegments(1).setFlush(false).get();
refresh(INDEX_NAME);
verifyStoreContent();
}
}
/**
* This test verifies that segment replication does not fail for closed indices
*/
public void testClosedIndices() {
List<String> nodes = new ArrayList<>();
// start 1st node so that it contains the primary
nodes.add(internalCluster().startDataOnlyNode());
createIndex(INDEX_NAME, super.indexSettings());
ensureYellowAndNoInitializingShards(INDEX_NAME);
// start 2nd node so that it contains the replica
nodes.add(internalCluster().startDataOnlyNode());
ensureGreen(INDEX_NAME);
logger.info("--> Close index");
assertAcked(client().admin().indices().prepareClose(INDEX_NAME));
logger.info("--> waiting for allocation to have shards assigned");
waitForRelocation(ClusterHealthStatus.GREEN);
}
/**
* This test validates the primary node drop does not result in shard failure on replica.
* @throws Exception when issue is encountered
*/
public void testNodeDropWithOngoingReplication() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(
INDEX_NAME,
Settings.builder()
.put(indexSettings())
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.put("index.refresh_interval", -1)
.build()
);
ensureYellow(INDEX_NAME);
final String replicaNode = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState();
// Get replica allocation id
final String replicaAllocationId = state.routingTable()
.index(INDEX_NAME)
.shardsWithState(ShardRoutingState.STARTED)
.stream()
.filter(routing -> routing.primary() == false)
.findFirst()
.get()
.allocationId()
.getId();
DiscoveryNode primaryDiscovery = state.nodes().resolveNode(primaryNode);
CountDownLatch blockFileCopy = new CountDownLatch(1);
MockTransportService primaryTransportService = ((MockTransportService) internalCluster().getInstance(
TransportService.class,
primaryNode
));
primaryTransportService.addSendBehavior(
internalCluster().getInstance(TransportService.class, replicaNode),
(connection, requestId, action, request, options) -> {
if (action.equals(SegmentReplicationTargetService.Actions.FILE_CHUNK)) {
FileChunkRequest req = (FileChunkRequest) request;
logger.debug("file chunk [{}] lastChunk: {}", req, req.lastChunk());
if (req.name().endsWith("cfs") && req.lastChunk()) {
try {
blockFileCopy.await();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
throw new NodeClosedException(primaryDiscovery);
}
}
connection.sendRequest(requestId, action, request, options);
}
);
final int docCount = scaledRandomIntBetween(1, 10);
for (int i = 0; i < docCount; i++) {
client().prepareIndex(INDEX_NAME).setId(Integer.toString(i)).setSource("field", "value" + i).execute().get();
}
// Refresh, this should trigger round of segment replication
refresh(INDEX_NAME);
blockFileCopy.countDown();
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
ensureYellow(INDEX_NAME);
assertBusy(() -> { assertDocCounts(docCount, replicaNode); });
state = client().admin().cluster().prepareState().execute().actionGet().getState();
// replica now promoted as primary should have same allocation id
final String currentAllocationID = state.routingTable()
.index(INDEX_NAME)
.shardsWithState(ShardRoutingState.STARTED)
.stream()
.filter(routing -> routing.primary())
.findFirst()
.get()
.allocationId()
.getId();
assertEquals(currentAllocationID, replicaAllocationId);
}
public void testCancellation() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build());
ensureYellow(INDEX_NAME);
final String replicaNode = internalCluster().startDataOnlyNode();
final SegmentReplicationSourceService segmentReplicationSourceService = internalCluster().getInstance(
SegmentReplicationSourceService.class,
primaryNode
);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
CountDownLatch latch = new CountDownLatch(1);
MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(
TransportService.class,
primaryNode
));
mockTransportService.addSendBehavior(
internalCluster().getInstance(TransportService.class, replicaNode),
(connection, requestId, action, request, options) -> {
if (action.equals(SegmentReplicationTargetService.Actions.FILE_CHUNK)) {
FileChunkRequest req = (FileChunkRequest) request;
logger.debug("file chunk [{}] lastChunk: {}", req, req.lastChunk());
if (req.name().endsWith("cfs") && req.lastChunk()) {
try {
latch.await();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
}
connection.sendRequest(requestId, action, request, options);
}
);
final int docCount = scaledRandomIntBetween(0, 10);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(docCount);
waitForDocs(docCount, indexer);
flush(INDEX_NAME);
}
segmentReplicationSourceService.beforeIndexShardClosed(primaryShard.shardId(), primaryShard, indexSettings());
latch.countDown();
assertDocCounts(docCount, primaryNode);
}
public void testStartReplicaAfterPrimaryIndexesDocs() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build());
ensureGreen(INDEX_NAME);
// Index a doc to create the first set of segments. _s1.si
client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").get();
// Flush segments to disk and create a new commit point (Primary: segments_3, _s1.si)
flushAndRefresh(INDEX_NAME);
assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 1);
// Index to create another segment
client().prepareIndex(INDEX_NAME).setId("2").setSource("foo", "bar").get();
// Force a merge here so that the in memory SegmentInfos does not reference old segments on disk.
client().admin().indices().prepareForceMerge(INDEX_NAME).setMaxNumSegments(1).setFlush(false).get();
refresh(INDEX_NAME);
assertAcked(
client().admin()
.indices()
.prepareUpdateSettings(INDEX_NAME)
.setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1))
);
final String replicaNode = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 2);
assertHitCount(client(replicaNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 2);
client().prepareIndex(INDEX_NAME).setId("3").setSource("foo", "bar").get();
refresh(INDEX_NAME);
waitForSearchableDocs(3, primaryNode, replicaNode);
assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 3);
assertHitCount(client(replicaNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 3);
verifyStoreContent();
}
@TestLogging(reason = "Getting trace logs from replication package", value = "org.opensearch.indices.replication:TRACE")
public void testDeleteOperations() throws Exception {
final String nodeA = internalCluster().startDataOnlyNode();
final String nodeB = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(1, 5);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, nodeA, nodeB);
final int additionalDocCount = scaledRandomIntBetween(0, 2);
final int expectedHitCount = initialDocCount + additionalDocCount;
indexer.start(additionalDocCount);
waitForDocs(expectedHitCount, indexer);
waitForSearchableDocs(expectedHitCount, nodeA, nodeB);
ensureGreen(INDEX_NAME);
Set<String> ids = indexer.getIds();
assertFalse(ids.isEmpty());
String id = ids.toArray()[0].toString();
client(nodeA).prepareDelete(INDEX_NAME, id).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
refresh(INDEX_NAME);
waitForSearchableDocs(expectedHitCount - 1, nodeA, nodeB);
verifyStoreContent();
}
}
/**
* This tests that the max seqNo we send to replicas is accurate and that after failover
* the new primary starts indexing from the correct maxSeqNo and replays the correct count of docs
* from xlog.
*/
public void testReplicationPostDeleteAndForceMerge() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(1, 10);
for (int i = 0; i < initialDocCount; i++) {
client().prepareIndex(INDEX_NAME).setId(String.valueOf(i)).setSource("foo", "bar").get();
}
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, primary, replica);
final int deletedDocCount = randomIntBetween(1, initialDocCount);
for (int i = 0; i < deletedDocCount; i++) {
client(primary).prepareDelete(INDEX_NAME, String.valueOf(i)).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
}
client().admin().indices().prepareForceMerge(INDEX_NAME).setMaxNumSegments(1).setFlush(false).get();
// randomly flush here after the force merge to wipe any old segments.
if (randomBoolean()) {
flush(INDEX_NAME);
}
final IndexShard primaryShard = getIndexShard(primary, INDEX_NAME);
final IndexShard replicaShard = getIndexShard(replica, INDEX_NAME);
assertBusy(
() -> assertEquals(
primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
)
);
// add some docs to the xlog and drop primary.
final int additionalDocs = randomIntBetween(1, 5);
for (int i = initialDocCount; i < initialDocCount + additionalDocs; i++) {
client().prepareIndex(INDEX_NAME).setId(String.valueOf(i)).setSource("foo", "bar").get();
}
// Drop the primary and wait until replica is promoted.
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary));
ensureYellowAndNoInitializingShards(INDEX_NAME);
final ShardRouting replicaShardRouting = getShardRoutingForNodeName(replica);
assertNotNull(replicaShardRouting);
assertTrue(replicaShardRouting + " should be promoted as a primary", replicaShardRouting.primary());
refresh(INDEX_NAME);
final long expectedHitCount = initialDocCount + additionalDocs - deletedDocCount;
assertHitCount(client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), expectedHitCount);
int expectedMaxSeqNo = initialDocCount + deletedDocCount + additionalDocs - 1;
assertEquals(expectedMaxSeqNo, replicaShard.seqNoStats().getMaxSeqNo());
// index another doc.
client().prepareIndex(INDEX_NAME).setId(String.valueOf(expectedMaxSeqNo + 1)).setSource("another", "doc").get();
refresh(INDEX_NAME);
assertHitCount(client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), expectedHitCount + 1);
}
public void testUpdateOperations() throws Exception {
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellow(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
final int initialDocCount = scaledRandomIntBetween(1, 5);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount, asList(primary, replica));
final int additionalDocCount = scaledRandomIntBetween(0, 5);
final int expectedHitCount = initialDocCount + additionalDocCount;
indexer.start(additionalDocCount);
waitForDocs(expectedHitCount, indexer);
waitForSearchableDocs(expectedHitCount, asList(primary, replica));
Set<String> ids = indexer.getIds();
String id = ids.toArray()[0].toString();
UpdateResponse updateResponse = client(primary).prepareUpdate(INDEX_NAME, id)
.setDoc(Requests.INDEX_CONTENT_TYPE, "foo", "baz")
.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL)
.get();
assertFalse("request shouldn't have forced a refresh", updateResponse.forcedRefresh());
assertEquals(2, updateResponse.getVersion());
refresh(INDEX_NAME);
verifyStoreContent();
assertSearchHits(client(primary).prepareSearch(INDEX_NAME).setQuery(matchQuery("foo", "baz")).get(), id);
assertSearchHits(client(replica).prepareSearch(INDEX_NAME).setQuery(matchQuery("foo", "baz")).get(), id);
}
}
public void testDropPrimaryDuringReplication() throws Exception {
final int replica_count = 6;
final Settings settings = Settings.builder()
.put(indexSettings())
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, replica_count)
.put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
.build();
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME, settings);
final List<String> dataNodes = internalCluster().startDataOnlyNodes(6);
ensureGreen(INDEX_NAME);
int initialDocCount = scaledRandomIntBetween(5, 10);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
// don't wait for replication to complete, stop the primary immediately.
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
ensureYellow(INDEX_NAME);
// start another replica.
dataNodes.add(internalCluster().startDataOnlyNode());
ensureGreen(INDEX_NAME);
waitForSearchableDocs(initialDocCount, dataNodes);
// index another doc and refresh - without this the new replica won't catch up.
String docId = String.valueOf(initialDocCount + 1);
client().prepareIndex(INDEX_NAME).setId(docId).setSource("foo", "bar").get();
flushAndRefresh(INDEX_NAME);
waitForSearchableDocs(initialDocCount + 1, dataNodes);
verifyStoreContent();
}
}
@TestLogging(reason = "Getting trace logs from replication package", value = "org.opensearch.indices.replication:TRACE")
public void testReplicaHasDiffFilesThanPrimary() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build());
ensureYellow(INDEX_NAME);
final String replicaNode = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
final IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
IndexWriterConfig iwc = newIndexWriterConfig().setOpenMode(IndexWriterConfig.OpenMode.APPEND);
// create a doc to index
int numDocs = 2 + random().nextInt(10);
List<Document> docs = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
doc.add(new StringField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
doc.add(
new TextField(
"body",
TestUtil.randomRealisticUnicodeString(random()),
random().nextBoolean() ? Field.Store.YES : Field.Store.NO
)
);
doc.add(new SortedDocValuesField("dv", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
docs.add(doc);
}
// create some segments on the replica before copy.
try (IndexWriter writer = new IndexWriter(replicaShard.store().directory(), iwc)) {
for (Document d : docs) {
writer.addDocument(d);
}
writer.flush();
writer.commit();
}
final SegmentInfos segmentInfos = SegmentInfos.readLatestCommit(replicaShard.store().directory());
replicaShard.finalizeReplication(segmentInfos);
ensureYellow(INDEX_NAME);
final int docCount = scaledRandomIntBetween(10, 20);
for (int i = 0; i < docCount; i++) {
client().prepareIndex(INDEX_NAME).setId(Integer.toString(i)).setSource("field", "value" + i).execute().get();
// Refresh, this should trigger round of segment replication
refresh(INDEX_NAME);
}
ensureGreen(INDEX_NAME);
waitForSearchableDocs(docCount, primaryNode, replicaNode);
verifyStoreContent();
final IndexShard replicaAfterFailure = getIndexShard(replicaNode, INDEX_NAME);
assertNotEquals(replicaAfterFailure.routingEntry().allocationId().getId(), replicaShard.routingEntry().allocationId().getId());
}
public void testPressureServiceStats() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellow(INDEX_NAME);
final String replicaNode = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
int initialDocCount = scaledRandomIntBetween(10, 20);
try (
BackgroundIndexer indexer = new BackgroundIndexer(
INDEX_NAME,
"_doc",
client(),
-1,
RandomizedTest.scaledRandomIntBetween(2, 5),
false,
random()
)
) {
indexer.start(initialDocCount);
waitForDocs(initialDocCount, indexer);
refresh(INDEX_NAME);
// get shard references.
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
final IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
logger.info("Replica aid {}", replicaShard.routingEntry().allocationId());
logger.info("former primary aid {}", primaryShard.routingEntry().allocationId());
// fetch pressure stats from the Primary's Node.
SegmentReplicationPressureService pressureService = internalCluster().getInstance(
SegmentReplicationPressureService.class,
primaryNode
);
// Fetch pressure stats from the Replica's Node we will assert replica node returns nothing until it is promoted.
SegmentReplicationPressureService replicaNode_service = internalCluster().getInstance(
SegmentReplicationPressureService.class,
replicaNode
);
final Map<ShardId, SegmentReplicationPerGroupStats> shardStats = pressureService.nodeStats().getShardStats();
assertEquals("We should have stats returned for the replication group", 1, shardStats.size());
SegmentReplicationPerGroupStats groupStats = shardStats.get(primaryShard.shardId());
Set<SegmentReplicationShardStats> replicaStats = groupStats.getReplicaStats();
assertAllocationIdsInReplicaShardStats(Set.of(replicaShard.routingEntry().allocationId().getId()), replicaStats);
assertTrue(replicaNode_service.nodeStats().getShardStats().isEmpty());
// drop the primary, this won't hand off pressure stats between old/new primary.
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
ensureYellowAndNoInitializingShards(INDEX_NAME);
assertTrue("replica should be promoted as a primary", replicaShard.routingEntry().primary());
assertEquals(
"We should have stats returned for the replication group",
1,
replicaNode_service.nodeStats().getShardStats().size()
);
// after the primary is dropped and replica is promoted we won't have a replica assigned yet, so stats per replica should return
// empty.
replicaStats = replicaNode_service.nodeStats().getShardStats().get(primaryShard.shardId()).getReplicaStats();
assertTrue(replicaStats.isEmpty());
// start another replica.
String replicaNode_2 = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
final IndexShard secondReplicaShard = getIndexShard(replicaNode_2, INDEX_NAME);
final String second_replica_aid = secondReplicaShard.routingEntry().allocationId().getId();
waitForSearchableDocs(initialDocCount, replicaNode_2);
assertEquals(
"We should have stats returned for the replication group",
1,
replicaNode_service.nodeStats().getShardStats().size()
);
replicaStats = replicaNode_service.nodeStats().getShardStats().get(replicaShard.shardId()).getReplicaStats();
assertAllocationIdsInReplicaShardStats(Set.of(second_replica_aid), replicaStats);
final SegmentReplicationShardStats replica_entry = replicaStats.stream().findFirst().get();
assertEquals(replica_entry.getCheckpointsBehindCount(), 0);
// test a checkpoint without any new segments
flush(INDEX_NAME);
assertBusy(() -> {
assertEquals(1, replicaNode_service.nodeStats().getShardStats().size());
final Set<SegmentReplicationShardStats> shardStatsSet = replicaNode_service.nodeStats()
.getShardStats()
.get(replicaShard.shardId())
.getReplicaStats();
assertAllocationIdsInReplicaShardStats(Set.of(second_replica_aid), shardStatsSet);
final SegmentReplicationShardStats stats = shardStatsSet.stream().findFirst().get();
assertEquals(0, stats.getCheckpointsBehindCount());
});
}
}
private void assertAllocationIdsInReplicaShardStats(Set<String> expected, Set<SegmentReplicationShardStats> replicaStats) {
assertEquals(expected, replicaStats.stream().map(SegmentReplicationShardStats::getAllocationId).collect(Collectors.toSet()));
}
/**
* Tests a scroll query on the replica
* @throws Exception when issue is encountered
*/
public void testScrollCreatedOnReplica() throws Exception {
// create the cluster with one primary node containing primary shard and replica node containing replica shard
final String primary = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replica = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);
// index 10 docs