-
Notifications
You must be signed in to change notification settings - Fork 2k
/
Copy pathSegmentReplicationPressureIT.java
379 lines (338 loc) · 16.8 KB
/
SegmentReplicationPressureIT.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
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.index;
import org.opensearch.action.admin.indices.replication.SegmentReplicationStatsResponse;
import org.opensearch.action.bulk.BulkItemResponse;
import org.opensearch.action.bulk.BulkRequest;
import org.opensearch.action.bulk.BulkResponse;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.UUIDs;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.lease.Releasable;
import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.IndexShardState;
import org.opensearch.indices.replication.SegmentReplicationBaseIT;
import org.opensearch.indices.replication.common.ReplicationType;
import org.opensearch.plugins.Plugin;
import org.opensearch.core.rest.RestStatus;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.transport.MockTransportService;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import static java.util.Arrays.asList;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
import static org.opensearch.index.SegmentReplicationPressureService.MAX_INDEXING_CHECKPOINTS;
import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_TIME_SETTING;
import static org.opensearch.index.SegmentReplicationPressureService.SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class SegmentReplicationPressureIT extends SegmentReplicationBaseIT {
private static final int MAX_CHECKPOINTS_BEHIND = 2;
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueSeconds(1))
.put(MAX_INDEXING_CHECKPOINTS.getKey(), MAX_CHECKPOINTS_BEHIND)
.build();
}
@Override
public Settings indexSettings() {
// we want to control refreshes
return Settings.builder()
.put(super.indexSettings())
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, SHARD_COUNT)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, REPLICA_COUNT)
.put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
.put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
.put("index.refresh_interval", -1)
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return asList(MockTransportService.TestPlugin.class);
}
public void testWritesRejected() throws Exception {
final String primaryNode = internalCluster().startNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replicaNode = internalCluster().startNode();
ensureGreen(INDEX_NAME);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
final List<String> replicaNodes = asList(replicaNode);
assertEqualSegmentInfosVersion(replicaNodes, primaryShard);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicInteger totalDocs = new AtomicInteger(0);
try (final Releasable ignored = blockReplication(replicaNodes, latch)) {
Thread indexingThread = new Thread(() -> { totalDocs.getAndSet(indexUntilCheckpointCount()); });
indexingThread.start();
indexingThread.join();
latch.await();
indexDoc();
totalDocs.incrementAndGet();
refresh(INDEX_NAME);
// index again while we are stale.
assertBusy(() -> {
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
indexDoc();
totalDocs.incrementAndGet();
});
});
// Try to index one more doc.
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
indexDoc();
totalDocs.incrementAndGet();
refresh(INDEX_NAME);
});
// Verify the rejected doc count.
SegmentReplicationStatsResponse segmentReplicationStatsResponse = dataNodeClient().admin()
.indices()
.prepareSegmentReplicationStats(INDEX_NAME)
.setDetailed(true)
.execute()
.actionGet();
SegmentReplicationPerGroupStats perGroupStats = segmentReplicationStatsResponse.getReplicationStats().get(INDEX_NAME).get(0);
assertEquals(perGroupStats.getRejectedRequestCount(), 2L);
}
refresh(INDEX_NAME);
// wait for the replicas to catch up after block is released.
assertReplicaCheckpointUpdated(primaryShard);
// index another doc showing there is no pressure enforced.
indexDoc();
refresh(INDEX_NAME);
waitForSearchableDocs(totalDocs.incrementAndGet(), replicaNodes.toArray(new String[] {}));
verifyStoreContent();
}
/**
* This test ensures that a replica can be added while the index is under write block.
* Ensuring that only write requests are blocked.
*/
public void testAddReplicaWhileWritesBlocked() throws Exception {
final String primaryNode = internalCluster().startNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replicaNode = internalCluster().startNode();
ensureGreen(INDEX_NAME);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
final List<String> replicaNodes = new ArrayList<>();
replicaNodes.add(replicaNode);
assertEqualSegmentInfosVersion(replicaNodes, primaryShard);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicInteger totalDocs = new AtomicInteger(0);
try (final Releasable ignored = blockReplication(replicaNodes, latch)) {
Thread indexingThread = new Thread(() -> { totalDocs.getAndSet(indexUntilCheckpointCount()); });
indexingThread.start();
indexingThread.join();
latch.await();
indexDoc();
totalDocs.incrementAndGet();
refresh(INDEX_NAME);
// index again while we are stale.
assertBusy(() -> {
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
indexDoc();
totalDocs.incrementAndGet();
});
});
final String replica_2 = internalCluster().startNode();
assertAcked(
client().admin()
.indices()
.prepareUpdateSettings(INDEX_NAME)
.setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, 2))
);
ensureGreen(INDEX_NAME);
replicaNodes.add(replica_2);
waitForSearchableDocs(totalDocs.get(), replica_2);
}
refresh(INDEX_NAME);
// wait for the replicas to catch up after block is released.
assertReplicaCheckpointUpdated(primaryShard);
// index another doc showing there is no pressure enforced.
indexDoc();
refresh(INDEX_NAME);
waitForSearchableDocs(totalDocs.incrementAndGet(), replicaNodes.toArray(new String[] {}));
verifyStoreContent();
}
public void testBelowReplicaLimit() throws Exception {
final Settings settings = Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3).build();
final String primaryNode = internalCluster().startNode();
createIndex(INDEX_NAME, settings);
ensureYellowAndNoInitializingShards(INDEX_NAME);
List<String> replicaNodes = new ArrayList<>();
for (int i = 0; i < 3; i++) {
replicaNodes.add(internalCluster().startNode());
}
ensureGreen(INDEX_NAME);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
assertEqualSegmentInfosVersion(replicaNodes, primaryShard);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicInteger totalDocs = new AtomicInteger(0);
// only block a single replica, pressure should not get applied.
try (final Releasable ignored = blockReplication(replicaNodes.subList(0, 1), latch)) {
Thread indexingThread = new Thread(() -> totalDocs.getAndSet(indexUntilCheckpointCount()));
indexingThread.start();
indexingThread.join();
latch.await();
indexDoc();
totalDocs.incrementAndGet();
refresh(INDEX_NAME);
}
// index another doc showing there is no pressure enforced.
indexDoc();
refresh(INDEX_NAME);
waitForSearchableDocs(totalDocs.incrementAndGet(), replicaNodes.toArray(new String[] {}));
verifyStoreContent();
}
public void testFailStaleReplica() throws Exception {
Settings settings = Settings.builder().put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(500)).build();
// Starts a primary and replica node.
final String primaryNode = internalCluster().startNode(settings);
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replicaNode = internalCluster().startNode(settings);
ensureGreen(INDEX_NAME);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
final List<String> replicaNodes = asList(replicaNode);
assertEqualSegmentInfosVersion(replicaNodes, primaryShard);
IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicInteger totalDocs = new AtomicInteger(0);
try (final Releasable ignored = blockReplication(replicaNodes, latch)) {
// Index docs until replicas are staled.
totalDocs.getAndSet(indexUntilCheckpointCount());
latch.await();
// index again while we are stale.
indexDoc();
refresh(INDEX_NAME);
totalDocs.incrementAndGet();
// Verify that replica shard is closed.
assertBusy(() -> { assertTrue(replicaShard.state().equals(IndexShardState.CLOSED)); }, 1, TimeUnit.MINUTES);
}
ensureGreen(INDEX_NAME);
final IndexShard replicaAfterFailure = getIndexShard(replicaNode, INDEX_NAME);
// Verify that new replica shard after failure is different from old replica shard.
assertNotEquals(replicaAfterFailure.routingEntry().allocationId().getId(), replicaShard.routingEntry().allocationId().getId());
}
public void testWithDocumentReplicationEnabledIndex() throws Exception {
assumeTrue(
"Can't create DocRep index with remote store enabled. Skipping.",
indexSettings().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false) == false
);
Settings settings = Settings.builder().put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(500)).build();
// Starts a primary and replica node.
final String primaryNode = internalCluster().startNode(settings);
createIndex(
INDEX_NAME,
Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT).build()
);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replicaNode = internalCluster().startNode(settings);
ensureGreen(INDEX_NAME);
final AtomicInteger totalDocs = new AtomicInteger(0);
// Index docs until replica stale limit is reached.
totalDocs.getAndSet(indexUntilCheckpointCount());
// index again after stale limit.
indexDoc();
refresh(INDEX_NAME);
totalDocs.incrementAndGet();
// verify total doc count is same and docs are not rejected.
assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), totalDocs.get());
assertHitCount(client(replicaNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), totalDocs.get());
}
public void testBulkWritesRejected() throws Exception {
final String primaryNode = internalCluster().startNode();
createIndex(INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
final String replicaNode = internalCluster().startNode();
final String coordinator = internalCluster().startCoordinatingOnlyNode(Settings.EMPTY);
ensureGreen(INDEX_NAME);
final IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
final List<String> replicaNodes = asList(replicaNode);
assertEqualSegmentInfosVersion(replicaNodes, primaryShard);
final CountDownLatch latch = new CountDownLatch(1);
List<String> nodes = List.of(primaryNode, replicaNode, coordinator);
int docsPerBatch = randomIntBetween(1, 200);
int totalDocs = docsPerBatch * MAX_CHECKPOINTS_BEHIND;
try (final Releasable ignored = blockReplication(replicaNodes, latch)) {
Thread indexingThread = new Thread(() -> {
for (int i = 0; i < MAX_CHECKPOINTS_BEHIND + 1; i++) {
executeBulkRequest(nodes, docsPerBatch);
refresh(INDEX_NAME);
}
});
indexingThread.start();
indexingThread.join();
latch.await();
// try and index again while we are stale.
assertBusy(() -> { assertFailedRequests(executeBulkRequest(nodes, randomIntBetween(1, 200))); });
}
refresh(INDEX_NAME);
// wait for the replicas to catch up after block is released.
assertReplicaCheckpointUpdated(primaryShard);
// index another doc showing there is no pressure enforced.
executeBulkRequest(nodes, totalDocs);
waitForSearchableDocs(totalDocs * 2L, replicaNodes.toArray(new String[] {}));
verifyStoreContent();
}
private BulkResponse executeBulkRequest(List<String> nodes, int docsPerBatch) {
final BulkRequest bulkRequest = new BulkRequest();
for (int j = 0; j < docsPerBatch; ++j) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
bulkRequest.add(request);
}
final BulkResponse bulkItemResponses = client(randomFrom(nodes)).bulk(bulkRequest).actionGet();
refresh(INDEX_NAME);
return bulkItemResponses;
}
/**
* Index and Refresh in batches to force checkpoints behind.
* Asserts that there are no stale replicas according to the primary until cp count is reached.
*/
private int indexUntilCheckpointCount() {
int total = 0;
for (int i = 0; i < MAX_CHECKPOINTS_BEHIND; i++) {
final int numDocs = randomIntBetween(1, 100);
for (int j = 0; j < numDocs; ++j) {
indexDoc();
}
total += numDocs;
refresh(INDEX_NAME);
}
return total;
}
private void assertFailedRequests(BulkResponse response) {
assertTrue(Arrays.stream(response.getItems()).allMatch(BulkItemResponse::isFailed));
assertTrue(
Arrays.stream(response.getItems())
.map(BulkItemResponse::getFailure)
.allMatch((failure) -> failure.getStatus() == RestStatus.TOO_MANY_REQUESTS)
);
}
private void indexDoc() {
client().prepareIndex(INDEX_NAME).setId(UUIDs.base64UUID()).setSource("{}", "{}").execute().actionGet();
}
private void assertEqualSegmentInfosVersion(List<String> replicaNames, IndexShard primaryShard) {
for (String replicaName : replicaNames) {
final IndexShard replicaShard = getIndexShard(replicaName, INDEX_NAME);
assertEquals(
primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
);
}
}
}