Skip to content

Commit

Permalink
Specialize pre-closing checks for engine implementations (#38702) (#3…
Browse files Browse the repository at this point in the history
…8727)

The Close Index API has been refactored in 6.7.0 and it now performs
pre-closing sanity checks on shards before an index is closed: the maximum
sequence number must be equals to the global checkpoint. While this is a
strong requirement for regular shards, we identified the need to relax this
check in the case of CCR following shards.

The following shards are not in charge of managing the max sequence
number or global checkpoint, which are pulled from a leader shard. They
also fetch and process batches of operations from the leader in an unordered
way, potentially leaving gaps in the history of ops. If the following shard lags
a lot it's possible that the global checkpoint and max seq number never get
in sync, preventing the following shard to be closed and a new PUT Follow
action to be issued on this shard (which is our recommended way to
resume/restart a CCR following).

This commit allows each Engine implementation to define the specific
verification it must perform before closing the index. In order to allow
following/frozen/closed shards to be closed whatever the max seq number
or global checkpoint are, the FollowingEngine and ReadOnlyEngine do
not perform any check before the index is closed.

Co-authored-by: Martijn van Groningen <martijn.v.groningen@gmail.com>

This commit also contains #37426.
Related #33888
  • Loading branch information
tlrx authored Feb 11, 2019
1 parent 232d2e0 commit 4b38523
Show file tree
Hide file tree
Showing 9 changed files with 246 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -108,13 +108,7 @@ private void executeShardOperation(final ShardRequest request, final IndexShard
if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), request.clusterBlock()) == false) {
throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + request.clusterBlock() + " before closing");
}

final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo();
if (indexShard.getGlobalCheckpoint() != maxSeqNo) {
throw new IllegalStateException("Global checkpoint [" + indexShard.getGlobalCheckpoint()
+ "] mismatches maximum sequence number [" + maxSeqNo + "] on index shard " + shardId);
}

indexShard.verifyShardBeforeIndexClosing();
indexShard.flush(new FlushRequest().force(true));
logger.trace("{} shard is ready for closing", shardId);
}
Expand Down
14 changes: 14 additions & 0 deletions server/src/main/java/org/elasticsearch/index/engine/Engine.java
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,20 @@ protected final DocsStats docsStats(IndexReader indexReader) {
return new DocsStats(numDocs, numDeletedDocs, sizeInBytes);
}

/**
* Performs the pre-closing checks on the {@link Engine}.
*
* @throws IllegalStateException if the sanity checks failed
*/
public void verifyEngineBeforeIndexClosing() throws IllegalStateException {
final long globalCheckpoint = engineConfig.getGlobalCheckpointSupplier().getAsLong();
final long maxSeqNo = getSeqNoStats(globalCheckpoint).getMaxSeqNo();
if (globalCheckpoint != maxSeqNo) {
throw new IllegalStateException("Global checkpoint [" + globalCheckpoint
+ "] mismatches maximum sequence number [" + maxSeqNo + "] on index shard " + shardId);
}
}

/**
* A throttling class that can be activated, causing the
* {@code acquireThrottle} method to block on a lock when throttling
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.Lock;
import org.elasticsearch.Assertions;
import org.elasticsearch.Version;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.core.internal.io.IOUtils;
Expand Down Expand Up @@ -98,7 +100,25 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats
indexWriterLock = obtainLock ? directory.obtainLock(IndexWriter.WRITE_LOCK_NAME) : null;
this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(directory);
this.translogStats = translogStats == null ? new TranslogStats(0, 0, 0, 0, 0) : translogStats;
this.seqNoStats = seqNoStats == null ? buildSeqNoStats(lastCommittedSegmentInfos) : seqNoStats;
if (seqNoStats == null) {
seqNoStats = buildSeqNoStats(lastCommittedSegmentInfos);
// During a peer-recovery the global checkpoint is not known and up to date when the engine
// is created, so we only check the max seq no / global checkpoint coherency when the global
// checkpoint is different from the unassigned sequence number value.
// In addition to that we only execute the check if the index the engine belongs to has been
// created after the refactoring of the Close Index API and its TransportVerifyShardBeforeCloseAction
// that guarantee that all operations have been flushed to Lucene.
final long globalCheckpoint = engineConfig.getGlobalCheckpointSupplier().getAsLong();
if (globalCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO
&& engineConfig.getIndexSettings().getIndexVersionCreated().onOrAfter(Version.V_6_7_0)) {
if (seqNoStats.getMaxSeqNo() != globalCheckpoint) {
assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), globalCheckpoint);
throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo()
+ "] from last commit does not match global checkpoint [" + globalCheckpoint + "]");
}
}
}
this.seqNoStats = seqNoStats;
this.indexCommit = Lucene.getIndexCommit(lastCommittedSegmentInfos, directory);
reader = open(indexCommit);
reader = wrapReader(reader, readerWrapperFunction);
Expand All @@ -116,6 +136,22 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats
}
}

protected void assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) {
if (Assertions.ENABLED) {
assert false : "max seq. no. [" + maxSeqNo + "] does not match [" + globalCheckpoint + "]";
}
}

@Override
public void verifyEngineBeforeIndexClosing() throws IllegalStateException {
// the value of the global checkpoint is verified when the read-only engine is opened,
// and it is not expected to change during the lifecycle of the engine. We could also
// check this value before closing the read-only engine but if something went wrong
// and the global checkpoint is not in-sync with the max. sequence number anymore,
// checking the value here again would prevent the read-only engine to be closed and
// reopened as an internal engine, which would be the path to fix the issue.
}

protected final DirectoryReader wrapReader(DirectoryReader reader,
Function<DirectoryReader, DirectoryReader> readerWrapperFunction) throws IOException {
reader = ElasticsearchDirectoryReader.wrap(reader, engineConfig.getShardId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3062,4 +3062,13 @@ public void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) {
getEngine().advanceMaxSeqNoOfUpdatesOrDeletes(seqNo);
assert seqNo <= getMaxSeqNoOfUpdatesOrDeletes() : getMaxSeqNoOfUpdatesOrDeletes() + " < " + seqNo;
}

/**
* Performs the pre-closing checks on the {@link IndexShard}.
*
* @throws IllegalStateException if the sanity checks failed
*/
public void verifyShardBeforeIndexClosing() throws IllegalStateException {
getEngine().verifyEngineBeforeIndexClosing();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,6 @@
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.seqno.SeqNoStats;
import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ReplicationGroup;
import org.elasticsearch.index.shard.ShardId;
Expand Down Expand Up @@ -73,6 +71,7 @@
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
Expand Down Expand Up @@ -100,8 +99,6 @@ public void setUp() throws Exception {

indexShard = mock(IndexShard.class);
when(indexShard.getActiveOperationsCount()).thenReturn(0);
when(indexShard.getGlobalCheckpoint()).thenReturn(0L);
when(indexShard.seqNoStats()).thenReturn(new SeqNoStats(0L, 0L, 0L));

final ShardId shardId = new ShardId("index", "_na_", randomIntBetween(0, 3));
when(indexShard.shardId()).thenReturn(shardId);
Expand Down Expand Up @@ -174,17 +171,16 @@ public void testOperationFailsWithNoBlock() {
verify(indexShard, times(0)).flush(any(FlushRequest.class));
}

public void testOperationFailsWithGlobalCheckpointNotCaughtUp() {
final long maxSeqNo = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, Long.MAX_VALUE);
final long localCheckpoint = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, maxSeqNo);
final long globalCheckpoint = randomValueOtherThan(maxSeqNo,
() -> randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, localCheckpoint));
when(indexShard.seqNoStats()).thenReturn(new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint));
when(indexShard.getGlobalCheckpoint()).thenReturn(globalCheckpoint);
public void testVerifyShardBeforeIndexClosing() throws Exception {
executeOnPrimaryOrReplica();
verify(indexShard, times(1)).verifyShardBeforeIndexClosing();
verify(indexShard, times(1)).flush(any(FlushRequest.class));
}

IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica);
assertThat(exception.getMessage(), equalTo("Global checkpoint [" + globalCheckpoint + "] mismatches maximum sequence number ["
+ maxSeqNo + "] on index shard " + indexShard.shardId()));
public void testVerifyShardBeforeIndexClosingFailed() {
doThrow(new IllegalStateException("test")).when(indexShard).verifyShardBeforeIndexClosing();
expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica);
verify(indexShard, times(1)).verifyShardBeforeIndexClosing();
verify(indexShard, times(0)).flush(any(FlushRequest.class));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public void testReadOnlyEngine() throws Exception {
lastDocIds = getDocIds(engine, true);
assertThat(readOnlyEngine.getLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
assertThat(readOnlyEngine.getSeqNoStats(globalCheckpoint.get()).getMaxSeqNo(), equalTo(lastSeqNoStats.getMaxSeqNo()));
assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds));
assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds));
for (int i = 0; i < numDocs; i++) {
if (randomBoolean()) {
String delId = Integer.toString(i);
Expand Down Expand Up @@ -126,7 +126,7 @@ public void testFlushes() throws IOException {
if (rarely()) {
engine.flush();
}
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
globalCheckpoint.set(i);
}
engine.syncTranslog();
engine.flushAndClose();
Expand All @@ -139,6 +139,40 @@ public void testFlushes() throws IOException {
}
}

public void testEnsureMaxSeqNoIsEqualToGlobalCheckpoint() throws IOException {
IOUtils.close(engine, store);
Engine readOnlyEngine = null;
final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
try (Store store = createStore()) {
EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get);
final int numDocs = scaledRandomIntBetween(10, 100);
try (InternalEngine engine = createEngine(config)) {
long maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED;
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null);
engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, VersionType.EXTERNAL,
Engine.Operation.Origin.REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0));
maxSeqNo = engine.getLocalCheckpoint();
}
globalCheckpoint.set(engine.getLocalCheckpoint() - 1);
engine.syncTranslog();
engine.flushAndClose();

IllegalStateException exception = expectThrows(IllegalStateException.class,
() -> new ReadOnlyEngine(engine.engineConfig, null, null, true, Function.identity()) {
@Override
protected void assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) {
// we don't want the assertion to trip in this test
}
});
assertThat(exception.getMessage(), equalTo("Maximum sequence number [" + maxSeqNo
+ "] from last commit does not match global checkpoint [" + globalCheckpoint.get() + "]"));
} finally {
IOUtils.close(readOnlyEngine);
}
}
}

public void testReadOnly() throws IOException {
IOUtils.close(engine, store);
final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
Expand All @@ -155,4 +189,25 @@ public void testReadOnly() throws IOException {
}
}
}

/**
* Test that {@link ReadOnlyEngine#verifyEngineBeforeIndexClosing()} never fails
* whatever the value of the global checkpoint to check is.
*/
public void testVerifyShardBeforeIndexClosingIsNoOp() throws IOException {
IOUtils.close(engine, store);
final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
try (Store store = createStore()) {
EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get);
store.createEmpty();
try (ReadOnlyEngine readOnlyEngine = new ReadOnlyEngine(config, null , null, true, Function.identity())) {
globalCheckpoint.set(randomNonNegativeLong());
try {
readOnlyEngine.verifyEngineBeforeIndexClosing();
} catch (final IllegalStateException e) {
fail("Read-only engine pre-closing verifications failed");
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -195,4 +195,12 @@ private OptionalLong lookupPrimaryTerm(final long seqNo) throws IOException {
public long getNumberOfOptimizedIndexing() {
return numOfOptimizedIndexing.count();
}

@Override
public void verifyEngineBeforeIndexClosing() throws IllegalStateException {
// the value of the global checkpoint is not verified when the following engine is closed,
// allowing it to be closed even in the case where all operations have not been fetched and
// processed from the leader and the operations history has gaps. This way the following
// engine can be closed and reopened in order to bootstrap the follower index again.
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ccr;

import org.elasticsearch.action.admin.indices.close.CloseIndexRequest;
import org.elasticsearch.action.admin.indices.open.OpenIndexRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.xpack.CcrIntegTestCase;
import org.elasticsearch.xpack.core.ccr.action.PutFollowAction;

import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;

import static java.util.Collections.singletonMap;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;

public class CloseFollowerIndexIT extends CcrIntegTestCase {

public void testCloseAndReopenFollowerIndex() throws Exception {
final String leaderIndexSettings = getIndexSettings(1, 1, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"));
assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON));
ensureLeaderYellow("index1");

PutFollowAction.Request followRequest = new PutFollowAction.Request();
followRequest.setRemoteCluster("leader_cluster");
followRequest.setLeaderIndex("index1");
followRequest.setFollowerIndex("index2");
followRequest.getParameters().setMaxRetryDelay(TimeValue.timeValueMillis(10));
followRequest.getParameters().setReadPollTimeout(TimeValue.timeValueMillis(10));
followRequest.getParameters().setMaxReadRequestSize(new ByteSizeValue(1));
followRequest.getParameters().setMaxOutstandingReadRequests(128);
followRequest.waitForActiveShards(ActiveShardCount.DEFAULT);

followerClient().execute(PutFollowAction.INSTANCE, followRequest).get();
ensureFollowerGreen("index2");

AtomicBoolean isRunning = new AtomicBoolean(true);
int numThreads = 4;
Thread[] threads = new Thread[numThreads];
for (int i = 0; i < numThreads; i++) {
threads[i] = new Thread(() -> {
while (isRunning.get()) {
leaderClient().prepareIndex("index1", "doc").setSource("{}", XContentType.JSON).get();
}
});
threads[i].start();
}

atLeastDocsIndexed(followerClient(), "index2", 32);
AcknowledgedResponse response = followerClient().admin().indices().close(new CloseIndexRequest("index2")).get();
assertThat(response.isAcknowledged(), is(true));

ClusterState clusterState = followerClient().admin().cluster().prepareState().get().getState();
List<ClusterBlock> blocks = new ArrayList<>(clusterState.getBlocks().indices().get("index2"));
assertThat(blocks.size(), equalTo(1));
assertThat(blocks.get(0).id(), equalTo(MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID));

isRunning.set(false);
for (Thread thread : threads) {
thread.join();
}
assertAcked(followerClient().admin().indices().open(new OpenIndexRequest("index2")).get());

refresh(leaderClient(), "index1");
SearchRequest leaderSearchRequest = new SearchRequest("index1");
leaderSearchRequest.source().trackTotalHits(true);
long leaderIndexDocs = leaderClient().search(leaderSearchRequest).actionGet().getHits().getTotalHits();
assertBusy(() -> {
refresh(followerClient(), "index2");
SearchRequest followerSearchRequest = new SearchRequest("index2");
followerSearchRequest.source().trackTotalHits(true);
long followerIndexDocs = followerClient().search(followerSearchRequest).actionGet().getHits().getTotalHits();
assertThat(followerIndexDocs, equalTo(leaderIndexDocs));
});
}
}
Loading

0 comments on commit 4b38523

Please sign in to comment.