Skip to content

Commit 0151974

Browse files
authored
_flush should block by default (#20597)
This commit changes the default behavior of `_flush` to block if other flushes are ongoing. This also removes the use of `FlushNotAllowedException` and instead simply return immediately by skipping the flush. Users should be aware if they set this option that the flush might or might not flush everything to disk ie. no transactional behavior of some sort. Closes #20569
1 parent 6dc03ec commit 0151974

File tree

24 files changed

+75
-95
lines changed

24 files changed

+75
-95
lines changed

core/src/main/java/org/elasticsearch/ElasticsearchException.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -633,8 +633,7 @@ enum ElasticsearchExceptionHandle {
633633
org.elasticsearch.repositories.RepositoryMissingException::new, 107),
634634
DOCUMENT_SOURCE_MISSING_EXCEPTION(org.elasticsearch.index.engine.DocumentSourceMissingException.class,
635635
org.elasticsearch.index.engine.DocumentSourceMissingException::new, 109),
636-
FLUSH_NOT_ALLOWED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.FlushNotAllowedEngineException.class,
637-
org.elasticsearch.index.engine.FlushNotAllowedEngineException::new, 110),
636+
// 110 used to be FlushNotAllowedEngineException
638637
NO_CLASS_SETTINGS_EXCEPTION(org.elasticsearch.common.settings.NoClassSettingsException.class,
639638
org.elasticsearch.common.settings.NoClassSettingsException::new, 111),
640639
BIND_TRANSPORT_EXCEPTION(org.elasticsearch.transport.BindTransportException.class,

core/src/main/java/org/elasticsearch/action/admin/indices/flush/FlushRequest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@
4040
public class FlushRequest extends BroadcastRequest<FlushRequest> {
4141

4242
private boolean force = false;
43-
private boolean waitIfOngoing = false;
43+
private boolean waitIfOngoing = true;
4444

4545
/**
4646
* Constructs a new flush request against one or more indices. If nothing is provided, all indices will
@@ -61,6 +61,7 @@ public boolean waitIfOngoing() {
6161
/**
6262
* if set to <tt>true</tt> the flush will block
6363
* if a another flush operation is already running until the flush can be performed.
64+
* The default is <code>true</code>
6465
*/
6566
public FlushRequest waitIfOngoing(boolean waitIfOngoing) {
6667
this.waitIfOngoing = waitIfOngoing;

core/src/main/java/org/elasticsearch/index/engine/Engine.java

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1105,8 +1105,6 @@ public void flushAndClose() throws IOException {
11051105
logger.debug("flushing shard on close - this might take some time to sync files to disk");
11061106
try {
11071107
flush(); // TODO we might force a flush in the future since we have the write lock already even though recoveries are running.
1108-
} catch (FlushNotAllowedEngineException ex) {
1109-
logger.debug("flush not allowed during flushAndClose - skipping");
11101108
} catch (EngineClosedException ex) {
11111109
logger.debug("engine already closed - skipping flushAndClose");
11121110
}
@@ -1233,4 +1231,11 @@ public interface Warmer {
12331231
* This operation will close the engine if the recovery fails.
12341232
*/
12351233
public abstract Engine recoverFromTranslog() throws IOException;
1234+
1235+
/**
1236+
* Returns <code>true</code> iff this engine is currently recovering from translog.
1237+
*/
1238+
public boolean isRecovering() {
1239+
return false;
1240+
}
12361241
}

core/src/main/java/org/elasticsearch/index/engine/FlushNotAllowedEngineException.java

Lines changed: 0 additions & 45 deletions
This file was deleted.

core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java

Lines changed: 15 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -116,7 +116,7 @@ public class InternalEngine extends Engine {
116116
// incoming indexing ops to a single thread:
117117
private final AtomicInteger throttleRequestCount = new AtomicInteger();
118118
private final EngineConfig.OpenMode openMode;
119-
private final AtomicBoolean allowCommits = new AtomicBoolean(true);
119+
private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false);
120120
private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1);
121121
private final CounterMetric numVersionLookups = new CounterMetric();
122122
private final CounterMetric numIndexVersionsLookups = new CounterMetric();
@@ -163,8 +163,9 @@ public InternalEngine(EngineConfig engineConfig) throws EngineException {
163163
manager = createSearcherManager();
164164
this.searcherManager = manager;
165165
this.versionMap.setManager(searcherManager);
166+
assert pendingTranslogRecovery.get() == false : "translog recovery can't be pending before we set it";
166167
// don't allow commits until we are done with recovering
167-
allowCommits.compareAndSet(true, openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG);
168+
pendingTranslogRecovery.set(openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG);
168169
if (engineConfig.getRefreshListeners() != null) {
169170
searcherManager.addListener(engineConfig.getRefreshListeners());
170171
}
@@ -190,14 +191,14 @@ public InternalEngine recoverFromTranslog() throws IOException {
190191
if (openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
191192
throw new IllegalStateException("Can't recover from translog with open mode: " + openMode);
192193
}
193-
if (allowCommits.get()) {
194+
if (pendingTranslogRecovery.get() == false) {
194195
throw new IllegalStateException("Engine has already been recovered");
195196
}
196197
try {
197198
recoverFromTranslog(engineConfig.getTranslogRecoveryPerformer());
198199
} catch (Exception e) {
199200
try {
200-
allowCommits.set(false); // just play safe and never allow commits on this
201+
pendingTranslogRecovery.set(true); // just play safe and never allow commits on this see #ensureCanFlush
201202
failEngine("failed to recover from translog", e);
202203
} catch (Exception inner) {
203204
e.addSuppressed(inner);
@@ -221,8 +222,8 @@ private void recoverFromTranslog(TranslogRecoveryPerformer handler) throws IOExc
221222
}
222223
// flush if we recovered something or if we have references to older translogs
223224
// note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length.
224-
assert allowCommits.get() == false : "commits are allowed but shouldn't";
225-
allowCommits.set(true); // we are good - now we can commit
225+
assert pendingTranslogRecovery.get(): "translogRecovery is not pending but should be";
226+
pendingTranslogRecovery.set(false); // we are good - now we can commit
226227
if (opsRecovered > 0) {
227228
logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]",
228229
opsRecovered, translogGeneration == null ? null : translogGeneration.translogFileGeneration, translog.currentFileGeneration());
@@ -765,7 +766,7 @@ public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineExcepti
765766
flushLock.lock();
766767
logger.trace("acquired flush lock after blocking");
767768
} else {
768-
throw new FlushNotAllowedEngineException(shardId, "already flushing...");
769+
return new CommitId(lastCommittedSegmentInfos.getId());
769770
}
770771
} else {
771772
logger.trace("acquired flush lock immediately");
@@ -1287,8 +1288,8 @@ private void ensureCanFlush() {
12871288
// if we are in this stage we have to prevent flushes from this
12881289
// engine otherwise we might loose documents if the flush succeeds
12891290
// and the translog recover fails we we "commit" the translog on flush.
1290-
if (allowCommits.get() == false) {
1291-
throw new FlushNotAllowedEngineException(shardId, "flushes are disabled - pending translog recovery");
1291+
if (pendingTranslogRecovery.get()) {
1292+
throw new IllegalStateException(shardId.toString() + " flushes are disabled - pending translog recovery");
12921293
}
12931294
}
12941295

@@ -1349,4 +1350,9 @@ private boolean incrementIndexVersionLookup() {
13491350
boolean indexWriterHasDeletions() {
13501351
return indexWriter.hasDeletions();
13511352
}
1353+
1354+
@Override
1355+
public boolean isRecovering() {
1356+
return pendingTranslogRecovery.get();
1357+
}
13521358
}

core/src/main/java/org/elasticsearch/index/shard/IndexShard.java

Lines changed: 19 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -730,7 +730,12 @@ public CompletionStats completionStats(String... fields) {
730730
public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expectedCommitId) {
731731
verifyStartedOrRecovering();
732732
logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId);
733-
return getEngine().syncFlush(syncId, expectedCommitId);
733+
Engine engine = getEngine();
734+
if (engine.isRecovering()) {
735+
throw new IllegalIndexShardStateException(shardId(), state, "syncFlush is only allowed if the engine is not recovery" +
736+
" from translog");
737+
}
738+
return engine.syncFlush(syncId, expectedCommitId);
734739
}
735740

736741
public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException {
@@ -741,11 +746,16 @@ public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException
741746
}
742747
// we allows flush while recovering, since we allow for operations to happen
743748
// while recovering, and we want to keep the translog at bay (up to deletes, which
744-
// we don't gc).
749+
// we don't gc). Yet, we don't use flush internally to clear deletes and flush the indexwriter since
750+
// we use #writeIndexingBuffer for this now.
745751
verifyStartedOrRecovering();
746-
752+
Engine engine = getEngine();
753+
if (engine.isRecovering()) {
754+
throw new IllegalIndexShardStateException(shardId(), state, "flush is only allowed if the engine is not recovery" +
755+
" from translog");
756+
}
747757
long time = System.nanoTime();
748-
Engine.CommitId commitId = getEngine().flush(force, waitIfOngoing);
758+
Engine.CommitId commitId = engine.flush(force, waitIfOngoing);
749759
flushMetric.inc(System.nanoTime() - time);
750760
return commitId;
751761

@@ -1165,7 +1175,11 @@ public void checkIdle(long inactiveTimeNS) {
11651175
boolean wasActive = active.getAndSet(false);
11661176
if (wasActive) {
11671177
logger.debug("shard is now inactive");
1168-
indexEventListener.onShardInactive(this);
1178+
try {
1179+
indexEventListener.onShardInactive(this);
1180+
} catch (Exception e) {
1181+
logger.warn("failed to notify index event listener", e);
1182+
}
11691183
}
11701184
}
11711185
}

core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@
3131
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
3232
import org.elasticsearch.index.engine.Engine;
3333
import org.elasticsearch.index.engine.EngineClosedException;
34-
import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
3534
import org.elasticsearch.index.shard.IndexShard;
3635
import org.elasticsearch.index.shard.IndexShardState;
3736
import org.elasticsearch.index.shard.IndexingOperationListener;
@@ -52,7 +51,7 @@
5251
public class IndexingMemoryController extends AbstractComponent implements IndexingOperationListener, Closeable {
5352

5453
/** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */
55-
public static final Setting<ByteSizeValue> INDEX_BUFFER_SIZE_SETTING =
54+
public static final Setting<ByteSizeValue> INDEX_BUFFER_SIZE_SETTING =
5655
Setting.memorySizeSetting("indices.memory.index_buffer_size", "10%", Property.NodeScope);
5756

5857
/** Only applies when <code>indices.memory.index_buffer_size</code> is a %, to set a floor on the actual size in bytes (default: 48 MB). */
@@ -386,7 +385,7 @@ private void runUnlocked() {
386385
protected void checkIdle(IndexShard shard, long inactiveTimeNS) {
387386
try {
388387
shard.checkIdle(inactiveTimeNS);
389-
} catch (EngineClosedException | FlushNotAllowedEngineException e) {
388+
} catch (EngineClosedException e) {
390389
logger.trace("ignore exception while checking if shard {} is inactive", e, shard.shardId());
391390
}
392391
}

core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -757,7 +757,7 @@ public void testIds() {
757757
ids.put(107, org.elasticsearch.repositories.RepositoryMissingException.class);
758758
ids.put(108, null);
759759
ids.put(109, org.elasticsearch.index.engine.DocumentSourceMissingException.class);
760-
ids.put(110, org.elasticsearch.index.engine.FlushNotAllowedEngineException.class);
760+
ids.put(110, null); // FlushNotAllowedEngineException was removed in 5.0
761761
ids.put(111, org.elasticsearch.common.settings.NoClassSettingsException.class);
762762
ids.put(112, org.elasticsearch.transport.BindTransportException.class);
763763
ids.put(113, org.elasticsearch.rest.action.admin.indices.AliasesNotFoundException.class);

core/src/test/java/org/elasticsearch/action/admin/indices/flush/FlushBlocksIT.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,7 @@ public void testFlushWithBlocks() {
4949
for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE)) {
5050
try {
5151
enableIndexBlock("test", blockSetting);
52-
FlushResponse response = client().admin().indices().prepareFlush("test").setWaitIfOngoing(true).execute().actionGet();
52+
FlushResponse response = client().admin().indices().prepareFlush("test").execute().actionGet();
5353
assertNoFailures(response);
5454
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));
5555
} finally {
@@ -80,4 +80,4 @@ public void testFlushWithBlocks() {
8080
setClusterReadOnly(false);
8181
}
8282
}
83-
}
83+
}

core/src/test/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsRequestTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ public void setupIndex() {
5454
String id = Integer.toString(j);
5555
client().prepareIndex("test", "type1", id).setSource("text", "sometext").get();
5656
}
57-
client().admin().indices().prepareFlush("test").setWaitIfOngoing(true).get();
57+
client().admin().indices().prepareFlush("test").get();
5858
}
5959

6060
public void testBasic() {

0 commit comments

Comments
 (0)