-
Notifications
You must be signed in to change notification settings - Fork 1.8k
/
SegmentReplicationIndexShardTests.java
531 lines (463 loc) · 24.9 KB
/
SegmentReplicationIndexShardTests.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
/*
* 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.index.shard;
import org.junit.Assert;
import org.opensearch.OpenSearchException;
import org.opensearch.action.ActionListener;
import org.opensearch.action.delete.DeleteRequest;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.CancellableThreads;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.engine.DocIdSeqNoAndSource;
import org.opensearch.index.engine.InternalEngine;
import org.opensearch.index.engine.NRTReplicationEngine;
import org.opensearch.index.engine.NRTReplicationEngineFactory;
import org.opensearch.index.mapper.MapperService;
import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase;
import org.opensearch.index.store.Store;
import org.opensearch.index.store.StoreFileMetadata;
import org.opensearch.indices.recovery.RecoverySettings;
import org.opensearch.indices.replication.CheckpointInfoResponse;
import org.opensearch.indices.replication.GetSegmentFilesResponse;
import org.opensearch.indices.replication.SegmentReplicationSource;
import org.opensearch.indices.replication.SegmentReplicationSourceFactory;
import org.opensearch.indices.replication.SegmentReplicationState;
import org.opensearch.indices.replication.SegmentReplicationTarget;
import org.opensearch.indices.replication.SegmentReplicationTargetService;
import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher;
import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint;
import org.opensearch.indices.replication.common.CopyState;
import org.opensearch.indices.replication.common.ReplicationType;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static java.util.Arrays.asList;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class SegmentReplicationIndexShardTests extends OpenSearchIndexLevelReplicationTestCase {
private static final Settings settings = Settings.builder()
.put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
.build();
/**
* Test that latestReplicationCheckpoint returns null only for docrep enabled indices
*/
public void testReplicationCheckpointNullForDocRep() throws IOException {
Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, "DOCUMENT").put(Settings.EMPTY).build();
final IndexShard indexShard = newStartedShard(false, indexSettings);
assertNull(indexShard.getLatestReplicationCheckpoint());
closeShards(indexShard);
}
/**
* Test that latestReplicationCheckpoint returns ReplicationCheckpoint for segrep enabled indices
*/
public void testReplicationCheckpointNotNullForSegReb() throws IOException {
Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, "SEGMENT").put(Settings.EMPTY).build();
final IndexShard indexShard = newStartedShard(indexSettings);
final ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint();
assertNotNull(replicationCheckpoint);
closeShards(indexShard);
}
public void testSegmentReplication_Index_Update_Delete() throws Exception {
String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}";
try (ReplicationGroup shards = createGroup(2, settings, mappings, new NRTReplicationEngineFactory())) {
shards.startAll();
final IndexShard primaryShard = shards.getPrimary();
final int numDocs = randomIntBetween(100, 200);
for (int i = 0; i < numDocs; i++) {
shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON));
}
primaryShard.refresh("Test");
replicateSegments(primaryShard, shards.getReplicas());
shards.assertAllEqual(numDocs);
for (int i = 0; i < numDocs; i++) {
// randomly update docs.
if (randomBoolean()) {
shards.index(
new IndexRequest(index.getName()).id(String.valueOf(i)).source("{ \"foo\" : \"baz\" }", XContentType.JSON)
);
}
}
primaryShard.refresh("Test");
replicateSegments(primaryShard, shards.getReplicas());
shards.assertAllEqual(numDocs);
final List<DocIdSeqNoAndSource> docs = getDocIdAndSeqNos(primaryShard);
for (IndexShard shard : shards.getReplicas()) {
assertEquals(getDocIdAndSeqNos(shard), docs);
}
for (int i = 0; i < numDocs; i++) {
// randomly delete.
if (randomBoolean()) {
shards.delete(new DeleteRequest(index.getName()).id(String.valueOf(i)));
}
}
primaryShard.refresh("Test");
replicateSegments(primaryShard, shards.getReplicas());
final List<DocIdSeqNoAndSource> docsAfterDelete = getDocIdAndSeqNos(primaryShard);
for (IndexShard shard : shards.getReplicas()) {
assertEquals(getDocIdAndSeqNos(shard), docsAfterDelete);
}
}
}
public void testIgnoreShardIdle() throws Exception {
try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
final IndexShard primary = shards.getPrimary();
final IndexShard replica = shards.getReplicas().get(0);
final int numDocs = shards.indexDocs(randomInt(10));
primary.refresh("test");
replicateSegments(primary, shards.getReplicas());
shards.assertAllEqual(numDocs);
primary.scheduledRefresh();
replica.scheduledRefresh();
primary.awaitShardSearchActive(b -> assertFalse("A new RefreshListener should not be registered", b));
replica.awaitShardSearchActive(b -> assertFalse("A new RefreshListener should not be registered", b));
// Update the search_idle setting, this will put both shards into search idle.
Settings updatedSettings = Settings.builder()
.put(settings)
.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.ZERO)
.build();
primary.indexSettings().getScopedSettings().applySettings(updatedSettings);
replica.indexSettings().getScopedSettings().applySettings(updatedSettings);
primary.scheduledRefresh();
replica.scheduledRefresh();
// Shards without segrep will register a new RefreshListener on the engine and return true when registered,
// assert with segrep enabled that awaitShardSearchActive does not register a listener.
primary.awaitShardSearchActive(b -> assertFalse("A new RefreshListener should not be registered", b));
replica.awaitShardSearchActive(b -> assertFalse("A new RefreshListener should not be registered", b));
}
}
/**
* here we are starting a new primary shard in PrimaryMode and testing if the shard publishes checkpoint after refresh.
*/
public void testPublishCheckpointOnPrimaryMode() throws IOException {
final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class);
IndexShard shard = newStartedShard(true);
CheckpointRefreshListener refreshListener = new CheckpointRefreshListener(shard, mock);
refreshListener.afterRefresh(true);
// verify checkpoint is published
verify(mock, times(1)).publish(any());
closeShards(shard);
}
/**
* here we are starting a new primary shard in PrimaryMode initially and starting relocation handoff. Later we complete relocation handoff then shard is no longer
* in PrimaryMode, and we test if the shard does not publish checkpoint after refresh.
*/
public void testPublishCheckpointAfterRelocationHandOff() throws IOException {
final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class);
IndexShard shard = newStartedShard(true);
CheckpointRefreshListener refreshListener = new CheckpointRefreshListener(shard, mock);
String id = shard.routingEntry().allocationId().getId();
// Starting relocation handoff
shard.getReplicationTracker().startRelocationHandoff(id);
// Completing relocation handoff
shard.getReplicationTracker().completeRelocationHandoff();
refreshListener.afterRefresh(true);
// verify checkpoint is not published
verify(mock, times(0)).publish(any());
closeShards(shard);
}
public void testNRTReplicaPromotedAsPrimary() throws Exception {
try (ReplicationGroup shards = createGroup(2, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
IndexShard oldPrimary = shards.getPrimary();
final IndexShard nextPrimary = shards.getReplicas().get(0);
final IndexShard replica = shards.getReplicas().get(1);
// 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point.
final int numDocs = shards.indexDocs(randomInt(10));
// refresh and copy the segments over.
oldPrimary.refresh("Test");
replicateSegments(oldPrimary, shards.getReplicas());
// at this point both shards should have numDocs persisted and searchable.
assertDocCounts(oldPrimary, numDocs, numDocs);
for (IndexShard shard : shards.getReplicas()) {
assertDocCounts(shard, numDocs, numDocs);
}
// 2. Create ops that are in the replica's xlog, not in the index.
// index some more into both but don't replicate. replica will have only numDocs searchable, but should have totalDocs
// persisted.
final int totalDocs = numDocs + shards.indexDocs(randomInt(10));
assertDocCounts(oldPrimary, totalDocs, totalDocs);
for (IndexShard shard : shards.getReplicas()) {
assertDocCounts(shard, totalDocs, numDocs);
}
// promote the replica
shards.syncGlobalCheckpoint();
assertEquals(totalDocs, nextPrimary.translogStats().estimatedNumberOfOperations());
shards.promoteReplicaToPrimary(nextPrimary);
// close and start the oldPrimary as a replica.
oldPrimary.close("demoted", false);
oldPrimary.store().close();
oldPrimary = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId());
shards.recoverReplica(oldPrimary);
assertEquals(NRTReplicationEngine.class, oldPrimary.getEngine().getClass());
assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass());
assertDocCounts(nextPrimary, totalDocs, totalDocs);
assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations());
// refresh and push segments to our other replica.
nextPrimary.refresh("test");
replicateSegments(nextPrimary, asList(replica));
for (IndexShard shard : shards) {
assertConsistentHistoryBetweenTranslogAndLucene(shard);
}
final List<DocIdSeqNoAndSource> docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary());
for (IndexShard shard : shards.getReplicas()) {
assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery));
}
}
}
public void testReplicaPromotedWhileReplicating() throws Exception {
try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
final IndexShard oldPrimary = shards.getPrimary();
final IndexShard nextPrimary = shards.getReplicas().get(0);
final int numDocs = shards.indexDocs(randomInt(10));
oldPrimary.refresh("Test");
shards.syncGlobalCheckpoint();
final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class);
final SegmentReplicationTargetService targetService = newTargetService(sourceFactory);
SegmentReplicationSource source = new SegmentReplicationSource() {
@Override
public void getCheckpointMetadata(
long replicationId,
ReplicationCheckpoint checkpoint,
ActionListener<CheckpointInfoResponse> listener
) {
resolveCheckpointInfoResponseListener(listener, oldPrimary);
ShardRouting oldRouting = nextPrimary.shardRouting;
try {
shards.promoteReplicaToPrimary(nextPrimary);
} catch (IOException e) {
Assert.fail("Promotion should not fail");
}
targetService.shardRoutingChanged(nextPrimary, oldRouting, nextPrimary.shardRouting);
}
@Override
public void getSegmentFiles(
long replicationId,
ReplicationCheckpoint checkpoint,
List<StoreFileMetadata> filesToFetch,
Store store,
ActionListener<GetSegmentFilesResponse> listener
) {
listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList()));
}
};
when(sourceFactory.get(any())).thenReturn(source);
startReplicationAndAssertCancellation(nextPrimary, targetService);
// wait for replica to finish being promoted, and assert doc counts.
final CountDownLatch latch = new CountDownLatch(1);
nextPrimary.acquirePrimaryOperationPermit(new ActionListener<>() {
@Override
public void onResponse(Releasable releasable) {
latch.countDown();
}
@Override
public void onFailure(Exception e) {
throw new AssertionError(e);
}
}, ThreadPool.Names.GENERIC, "");
latch.await();
assertEquals(nextPrimary.getEngine().getClass(), InternalEngine.class);
nextPrimary.refresh("test");
oldPrimary.close("demoted", false);
oldPrimary.store().close();
IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId());
shards.recoverReplica(newReplica);
assertDocCount(nextPrimary, numDocs);
assertDocCount(newReplica, numDocs);
nextPrimary.refresh("test");
replicateSegments(nextPrimary, shards.getReplicas());
final List<DocIdSeqNoAndSource> docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary());
for (IndexShard shard : shards.getReplicas()) {
assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery));
}
}
}
public void testReplicaClosesWhileReplicating_AfterGetCheckpoint() throws Exception {
try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
IndexShard primary = shards.getPrimary();
final IndexShard replica = shards.getReplicas().get(0);
final int numDocs = shards.indexDocs(randomInt(10));
primary.refresh("Test");
final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class);
final SegmentReplicationTargetService targetService = newTargetService(sourceFactory);
SegmentReplicationSource source = new SegmentReplicationSource() {
@Override
public void getCheckpointMetadata(
long replicationId,
ReplicationCheckpoint checkpoint,
ActionListener<CheckpointInfoResponse> listener
) {
// trigger a cancellation by closing the replica.
targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY);
resolveCheckpointInfoResponseListener(listener, primary);
}
@Override
public void getSegmentFiles(
long replicationId,
ReplicationCheckpoint checkpoint,
List<StoreFileMetadata> filesToFetch,
Store store,
ActionListener<GetSegmentFilesResponse> listener
) {
Assert.fail("Should not be reached");
}
};
when(sourceFactory.get(any())).thenReturn(source);
startReplicationAndAssertCancellation(replica, targetService);
shards.removeReplica(replica);
closeShards(replica);
}
}
public void testReplicaClosesWhileReplicating_AfterGetSegmentFiles() throws Exception {
try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
IndexShard primary = shards.getPrimary();
final IndexShard replica = shards.getReplicas().get(0);
final int numDocs = shards.indexDocs(randomInt(10));
primary.refresh("Test");
final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class);
final SegmentReplicationTargetService targetService = newTargetService(sourceFactory);
SegmentReplicationSource source = new SegmentReplicationSource() {
@Override
public void getCheckpointMetadata(
long replicationId,
ReplicationCheckpoint checkpoint,
ActionListener<CheckpointInfoResponse> listener
) {
resolveCheckpointInfoResponseListener(listener, primary);
}
@Override
public void getSegmentFiles(
long replicationId,
ReplicationCheckpoint checkpoint,
List<StoreFileMetadata> filesToFetch,
Store store,
ActionListener<GetSegmentFilesResponse> listener
) {
// randomly resolve the listener, indicating the source has resolved.
listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList()));
targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY);
}
};
when(sourceFactory.get(any())).thenReturn(source);
startReplicationAndAssertCancellation(replica, targetService);
shards.removeReplica(replica);
closeShards(replica);
}
}
public void testPrimaryCancelsExecution() throws Exception {
try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) {
shards.startAll();
IndexShard primary = shards.getPrimary();
final IndexShard replica = shards.getReplicas().get(0);
final int numDocs = shards.indexDocs(randomInt(10));
primary.refresh("Test");
final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class);
final SegmentReplicationTargetService targetService = newTargetService(sourceFactory);
SegmentReplicationSource source = new SegmentReplicationSource() {
@Override
public void getCheckpointMetadata(
long replicationId,
ReplicationCheckpoint checkpoint,
ActionListener<CheckpointInfoResponse> listener
) {
listener.onFailure(new CancellableThreads.ExecutionCancelledException("Cancelled"));
}
@Override
public void getSegmentFiles(
long replicationId,
ReplicationCheckpoint checkpoint,
List<StoreFileMetadata> filesToFetch,
Store store,
ActionListener<GetSegmentFilesResponse> listener
) {}
};
when(sourceFactory.get(any())).thenReturn(source);
startReplicationAndAssertCancellation(replica, targetService);
shards.removeReplica(replica);
closeShards(replica);
}
}
private SegmentReplicationTargetService newTargetService(SegmentReplicationSourceFactory sourceFactory) {
return new SegmentReplicationTargetService(
threadPool,
new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)),
mock(TransportService.class),
sourceFactory
);
}
/**
* Assert persisted and searchable doc counts. This method should not be used while docs are concurrently indexed because
* it asserts point in time seqNos are relative to the doc counts.
*/
private void assertDocCounts(IndexShard indexShard, int expectedPersistedDocCount, int expectedSearchableDocCount) throws IOException {
assertDocCount(indexShard, expectedSearchableDocCount);
// assigned seqNos start at 0, so assert max & local seqNos are 1 less than our persisted doc count.
assertEquals(expectedPersistedDocCount - 1, indexShard.seqNoStats().getMaxSeqNo());
assertEquals(expectedPersistedDocCount - 1, indexShard.seqNoStats().getLocalCheckpoint());
// processed cp should be 1 less than our searchable doc count.
assertEquals(expectedSearchableDocCount - 1, indexShard.getProcessedLocalCheckpoint());
}
private void resolveCheckpointInfoResponseListener(ActionListener<CheckpointInfoResponse> listener, IndexShard primary) {
try {
final CopyState copyState = new CopyState(ReplicationCheckpoint.empty(primary.shardId), primary);
listener.onResponse(
new CheckpointInfoResponse(
copyState.getCheckpoint(),
copyState.getMetadataSnapshot(),
copyState.getInfosBytes(),
copyState.getPendingDeleteFiles()
)
);
} catch (IOException e) {
logger.error("Unexpected error computing CopyState", e);
Assert.fail("Failed to compute copyState");
}
}
private void startReplicationAndAssertCancellation(IndexShard replica, SegmentReplicationTargetService targetService)
throws InterruptedException {
CountDownLatch latch = new CountDownLatch(1);
final SegmentReplicationTarget target = targetService.startReplication(
ReplicationCheckpoint.empty(replica.shardId),
replica,
new SegmentReplicationTargetService.SegmentReplicationListener() {
@Override
public void onReplicationDone(SegmentReplicationState state) {
Assert.fail("Replication should not complete");
}
@Override
public void onReplicationFailure(SegmentReplicationState state, OpenSearchException e, boolean sendShardFailure) {
assertTrue(e instanceof CancellableThreads.ExecutionCancelledException);
assertFalse(sendShardFailure);
assertEquals(SegmentReplicationState.Stage.CANCELLED, state.getStage());
latch.countDown();
}
}
);
latch.await(2, TimeUnit.SECONDS);
assertEquals("Should have resolved listener with failure", 0, latch.getCount());
assertNull(targetService.get(target.getId()));
}
}