|
31 | 31 | import org.elasticsearch.common.xcontent.XContentType;
|
32 | 32 | import org.elasticsearch.index.IndexService;
|
33 | 33 | import org.elasticsearch.index.IndexSettings;
|
| 34 | +import org.elasticsearch.index.shard.IndexShard; |
34 | 35 | import org.elasticsearch.index.translog.Translog;
|
| 36 | +import org.elasticsearch.indices.IndicesService; |
35 | 37 | import org.elasticsearch.plugins.Plugin;
|
36 | 38 | import org.elasticsearch.test.ESIntegTestCase;
|
37 | 39 | import org.elasticsearch.test.InternalSettingsPlugin;
|
@@ -226,4 +228,35 @@ private void runGlobalCheckpointSyncTest(
|
226 | 228 | }
|
227 | 229 | }
|
228 | 230 |
|
| 231 | + public void testPersistGlobalCheckpoint() throws Exception { |
| 232 | + internalCluster().ensureAtLeastNumDataNodes(2); |
| 233 | + Settings.Builder indexSettings = Settings.builder() |
| 234 | + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), randomTimeValue(100, 1000, "ms")) |
| 235 | + .put("index.number_of_replicas", randomIntBetween(0, 1)); |
| 236 | + if (randomBoolean()) { |
| 237 | + indexSettings.put(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING.getKey(), Translog.Durability.ASYNC) |
| 238 | + .put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), randomTimeValue(100, 1000, "ms")); |
| 239 | + } |
| 240 | + prepareCreate("test", indexSettings).get(); |
| 241 | + if (randomBoolean()) { |
| 242 | + ensureGreen("test"); |
| 243 | + } |
| 244 | + int numDocs = randomIntBetween(1, 20); |
| 245 | + for (int i = 0; i < numDocs; i++) { |
| 246 | + client().prepareIndex("test", "test", Integer.toString(i)).setSource("{}", XContentType.JSON).get(); |
| 247 | + } |
| 248 | + ensureGreen("test"); |
| 249 | + assertBusy(() -> { |
| 250 | + for (IndicesService indicesService : internalCluster().getDataNodeInstances(IndicesService.class)) { |
| 251 | + for (IndexService indexService : indicesService) { |
| 252 | + for (IndexShard shard : indexService) { |
| 253 | + final SeqNoStats seqNoStats = shard.seqNoStats(); |
| 254 | + assertThat(seqNoStats.getLocalCheckpoint(), equalTo(seqNoStats.getMaxSeqNo())); |
| 255 | + assertThat(shard.getLastKnownGlobalCheckpoint(), equalTo(seqNoStats.getMaxSeqNo())); |
| 256 | + assertThat(shard.getLastSyncedGlobalCheckpoint(), equalTo(seqNoStats.getMaxSeqNo())); |
| 257 | + } |
| 258 | + } |
| 259 | + } |
| 260 | + }); |
| 261 | + } |
229 | 262 | }
|
0 commit comments