Skip to content

Commit b54a9e9

Browse files
authored
Introduce translog generation rolling
This commit introduces a maximum size for a translog generation and automatically rolls the translog when a generation exceeds the threshold into a new generation. This threshold is configurable per index and defaults to sixty-four megabytes. We introduce this constraint as sequence numbers will require keeping around more than the current generation (to ensure that we can rollback to the global checkpoint). Without keeping the size of generations under control, having to keep old generations around could consume excessive disk space. A follow-up will enable commits to trim previous generations based on the global checkpoint. Relates #23606
1 parent defd045 commit b54a9e9

File tree

9 files changed

+441
-113
lines changed

9 files changed

+441
-113
lines changed

core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -22,16 +22,12 @@
2222
import org.apache.logging.log4j.Logger;
2323
import org.apache.logging.log4j.message.ParameterizedMessage;
2424
import org.elasticsearch.action.ActionListener;
25-
import org.elasticsearch.action.bulk.BulkRequest;
26-
import org.elasticsearch.action.bulk.BulkShardRequest;
2725
import org.elasticsearch.action.support.ActionFilters;
2826
import org.elasticsearch.action.support.WriteRequest;
2927
import org.elasticsearch.action.support.WriteResponse;
30-
import org.elasticsearch.client.transport.NoNodeAvailableException;
3128
import org.elasticsearch.cluster.action.shard.ShardStateAction;
3229
import org.elasticsearch.cluster.block.ClusterBlockLevel;
3330
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
34-
import org.elasticsearch.cluster.node.DiscoveryNode;
3531
import org.elasticsearch.cluster.routing.ShardRouting;
3632
import org.elasticsearch.cluster.service.ClusterService;
3733
import org.elasticsearch.common.Nullable;
@@ -46,7 +42,6 @@
4642
import org.elasticsearch.transport.TransportException;
4743
import org.elasticsearch.transport.TransportResponse;
4844
import org.elasticsearch.transport.TransportService;
49-
import org.apache.logging.log4j.core.pattern.ConverterKeys;
5045

5146
import java.util.concurrent.atomic.AtomicBoolean;
5247
import java.util.concurrent.atomic.AtomicInteger;
@@ -302,15 +297,21 @@ private void maybeFinish() {
302297
}
303298

304299
void run() {
305-
// we either respond immediately ie. if we we don't fsync per request or wait for refresh
306-
// OR we got an pass async operations on and wait for them to return to respond.
307-
indexShard.maybeFlush();
308-
maybeFinish(); // decrement the pendingOpts by one, if there is nothing else to do we just respond with success.
300+
/*
301+
* We either respond immediately (i.e., if we do not fsync per request or wait for
302+
* refresh), or we there are past async operations and we wait for them to return to
303+
* respond.
304+
*/
305+
indexShard.afterWriteOperation();
306+
// decrement pending by one, if there is nothing else to do we just respond with success
307+
maybeFinish();
309308
if (waitUntilRefresh) {
310309
assert pendingOps.get() > 0;
311310
indexShard.addRefreshListener(location, forcedRefresh -> {
312311
if (forcedRefresh) {
313-
logger.warn("block_until_refresh request ran out of slots and forced a refresh: [{}]", request);
312+
logger.warn(
313+
"block until refresh ran out of slots and forced a refresh: [{}]",
314+
request);
314315
}
315316
refreshed.set(forcedRefresh);
316317
maybeFinish();

core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -125,6 +125,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
125125
EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING,
126126
EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING,
127127
IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING,
128+
IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING,
128129
IndexFieldDataService.INDEX_FIELDDATA_CACHE_KEY,
129130
FieldMapper.IGNORE_MALFORMED_SETTING,
130131
FieldMapper.COERCE_SETTING,

core/src/main/java/org/elasticsearch/index/IndexSettings.java

Lines changed: 32 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import org.apache.lucene.index.MergePolicy;
2323
import org.elasticsearch.Version;
2424
import org.elasticsearch.cluster.metadata.IndexMetaData;
25-
import org.elasticsearch.common.logging.DeprecationLogger;
2625
import org.elasticsearch.common.logging.Loggers;
2726
import org.elasticsearch.common.settings.IndexScopedSettings;
2827
import org.elasticsearch.common.settings.Setting;
@@ -112,6 +111,16 @@ public final class IndexSettings {
112111
Setting.byteSizeSetting("index.translog.flush_threshold_size", new ByteSizeValue(512, ByteSizeUnit.MB), Property.Dynamic,
113112
Property.IndexScope);
114113

114+
/**
115+
* The maximum size of a translog generation. This is independent of the maximum size of
116+
* translog operations that have not been flushed.
117+
*/
118+
public static final Setting<ByteSizeValue> INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING =
119+
Setting.byteSizeSetting(
120+
"index.translog.generation_threshold_size",
121+
new ByteSizeValue(64, ByteSizeUnit.MB),
122+
new Property[]{Property.Dynamic, Property.IndexScope});
123+
115124
public static final Setting<TimeValue> INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL =
116125
Setting.timeSetting("index.seq_no.checkpoint_sync_interval", new TimeValue(30, TimeUnit.SECONDS),
117126
new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope);
@@ -156,6 +165,7 @@ public final class IndexSettings {
156165
private volatile TimeValue refreshInterval;
157166
private final TimeValue globalCheckpointInterval;
158167
private volatile ByteSizeValue flushThresholdSize;
168+
private volatile ByteSizeValue generationThresholdSize;
159169
private final MergeSchedulerConfig mergeSchedulerConfig;
160170
private final MergePolicyConfig mergePolicyConfig;
161171
private final IndexScopedSettings scopedSettings;
@@ -250,6 +260,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti
250260
refreshInterval = scopedSettings.get(INDEX_REFRESH_INTERVAL_SETTING);
251261
globalCheckpointInterval = scopedSettings.get(INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL);
252262
flushThresholdSize = scopedSettings.get(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING);
263+
generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING);
253264
mergeSchedulerConfig = new MergeSchedulerConfig(this);
254265
gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis();
255266
warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING);
@@ -281,6 +292,9 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti
281292
scopedSettings.addSettingsUpdateConsumer(INDEX_WARMER_ENABLED_SETTING, this::setEnableWarmer);
282293
scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes);
283294
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, this::setTranslogFlushThresholdSize);
295+
scopedSettings.addSettingsUpdateConsumer(
296+
INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING,
297+
this::setGenerationThresholdSize);
284298
scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval);
285299
scopedSettings.addSettingsUpdateConsumer(MAX_REFRESH_LISTENERS_PER_SHARD, this::setMaxRefreshListeners);
286300
scopedSettings.addSettingsUpdateConsumer(MAX_SLICES_PER_SCROLL, this::setMaxSlicesPerScroll);
@@ -290,6 +304,10 @@ private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) {
290304
this.flushThresholdSize = byteSizeValue;
291305
}
292306

307+
private void setGenerationThresholdSize(final ByteSizeValue generationThresholdSize) {
308+
this.generationThresholdSize = generationThresholdSize;
309+
}
310+
293311
private void setGCDeletes(TimeValue timeValue) {
294312
this.gcDeletesInMillis = timeValue.getMillis();
295313
}
@@ -461,6 +479,19 @@ public TimeValue getGlobalCheckpointInterval() {
461479
*/
462480
public ByteSizeValue getFlushThresholdSize() { return flushThresholdSize; }
463481

482+
/**
483+
* Returns the generation threshold size. As sequence numbers can cause multiple generations to
484+
* be preserved for rollback purposes, we want to keep the size of individual generations from
485+
* growing too large to avoid excessive disk space consumption. Therefore, the translog is
486+
* automatically rolled to a new generation when the current generation exceeds this generation
487+
* threshold size.
488+
*
489+
* @return the generation threshold size
490+
*/
491+
public ByteSizeValue getGenerationThresholdSize() {
492+
return generationThresholdSize;
493+
}
494+
464495
/**
465496
* Returns the {@link MergeSchedulerConfig}
466497
*/

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

Lines changed: 108 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -771,27 +771,44 @@ public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expecte
771771
return engine.syncFlush(syncId, expectedCommitId);
772772
}
773773

774-
public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException {
775-
boolean waitIfOngoing = request.waitIfOngoing();
776-
boolean force = request.force();
777-
if (logger.isTraceEnabled()) {
778-
logger.trace("flush with {}", request);
779-
}
780-
// we allows flush while recovering, since we allow for operations to happen
781-
// while recovering, and we want to keep the translog at bay (up to deletes, which
782-
// we don't gc). Yet, we don't use flush internally to clear deletes and flush the indexwriter since
783-
// we use #writeIndexingBuffer for this now.
774+
/**
775+
* Executes the given flush request against the engine.
776+
*
777+
* @param request the flush request
778+
* @return the commit ID
779+
*/
780+
public Engine.CommitId flush(FlushRequest request) {
781+
final boolean waitIfOngoing = request.waitIfOngoing();
782+
final boolean force = request.force();
783+
logger.trace("flush with {}", request);
784+
/*
785+
* We allow flushes while recovery since we allow operations to happen while recovering and
786+
* we want to keep the translog under control (up to deletes, which we do not GC). Yet, we
787+
* do not use flush internally to clear deletes and flush the index writer since we use
788+
* Engine#writeIndexingBuffer for this now.
789+
*/
784790
verifyNotClosed();
785-
Engine engine = getEngine();
791+
final Engine engine = getEngine();
786792
if (engine.isRecovering()) {
787-
throw new IllegalIndexShardStateException(shardId(), state, "flush is only allowed if the engine is not recovery" +
788-
" from translog");
793+
throw new IllegalIndexShardStateException(
794+
shardId(),
795+
state,
796+
"flush is only allowed if the engine is not recovery from translog");
789797
}
790-
long time = System.nanoTime();
791-
Engine.CommitId commitId = engine.flush(force, waitIfOngoing);
798+
final long time = System.nanoTime();
799+
final Engine.CommitId commitId = engine.flush(force, waitIfOngoing);
792800
flushMetric.inc(System.nanoTime() - time);
793801
return commitId;
802+
}
794803

804+
/**
805+
* Rolls the tranlog generation.
806+
*
807+
* @throws IOException if any file operations on the translog throw an I/O exception
808+
*/
809+
private void rollTranslogGeneration() throws IOException {
810+
final Engine engine = getEngine();
811+
engine.getTranslog().rollGeneration();
795812
}
796813

797814
public void forceMerge(ForceMergeRequest forceMerge) throws IOException {
@@ -1256,17 +1273,39 @@ public boolean restoreFromRepository(Repository repository) {
12561273
}
12571274

12581275
/**
1259-
* Returns <code>true</code> iff this shard needs to be flushed due to too many translog operation or a too large transaction log.
1260-
* Otherwise <code>false</code>.
1276+
* Tests whether or not the translog should be flushed. This test is based on the current size
1277+
* of the translog comparted to the configured flush threshold size.
1278+
*
1279+
* @return {@code true} if the translog should be flushed
12611280
*/
12621281
boolean shouldFlush() {
1263-
Engine engine = getEngineOrNull();
1282+
final Engine engine = getEngineOrNull();
12641283
if (engine != null) {
12651284
try {
1266-
Translog translog = engine.getTranslog();
1267-
return translog.sizeInBytes() > indexSettings.getFlushThresholdSize().getBytes();
1268-
} catch (AlreadyClosedException ex) {
1269-
// that's fine we are already close - no need to flush
1285+
final Translog translog = engine.getTranslog();
1286+
return translog.shouldFlush();
1287+
} catch (final AlreadyClosedException e) {
1288+
// we are already closed, no need to flush or roll
1289+
}
1290+
}
1291+
return false;
1292+
}
1293+
1294+
/**
1295+
* Tests whether or not the translog generation should be rolled to a new generation. This test
1296+
* is based on the size of the current generation compared to the configured generation
1297+
* threshold size.
1298+
*
1299+
* @return {@code true} if the current generation should be rolled to a new generation
1300+
*/
1301+
boolean shouldRollTranslogGeneration() {
1302+
final Engine engine = getEngineOrNull();
1303+
if (engine != null) {
1304+
try {
1305+
final Translog translog = engine.getTranslog();
1306+
return translog.shouldRollGeneration();
1307+
} catch (final AlreadyClosedException e) {
1308+
// we are already closed, no need to flush or roll
12701309
}
12711310
}
12721311
return false;
@@ -1810,28 +1849,31 @@ public Translog.Durability getTranslogDurability() {
18101849
return indexSettings.getTranslogDurability();
18111850
}
18121851

1813-
private final AtomicBoolean asyncFlushRunning = new AtomicBoolean();
1852+
// we can not protect with a lock since we "release" on a different thread
1853+
private final AtomicBoolean flushOrRollRunning = new AtomicBoolean();
18141854

18151855
/**
1816-
* Schedules a flush if needed but won't schedule more than one flush concurrently. The flush will be executed on the
1817-
* Flush thread-pool asynchronously.
1818-
*
1819-
* @return <code>true</code> if a new flush is scheduled otherwise <code>false</code>.
1856+
* Schedules a flush or translog generation roll if needed but will not schedule more than one
1857+
* concurrently. The operation will be executed asynchronously on the flush thread pool.
18201858
*/
1821-
public boolean maybeFlush() {
1822-
if (shouldFlush()) {
1823-
if (asyncFlushRunning.compareAndSet(false, true)) { // we can't use a lock here since we "release" in a different thread
1824-
if (shouldFlush() == false) {
1825-
// we have to check again since otherwise there is a race when a thread passes
1826-
// the first shouldFlush() check next to another thread which flushes fast enough
1827-
// to finish before the current thread could flip the asyncFlushRunning flag.
1828-
// in that situation we have an extra unexpected flush.
1829-
asyncFlushRunning.compareAndSet(true, false);
1830-
} else {
1859+
public void afterWriteOperation() {
1860+
if (shouldFlush() || shouldRollTranslogGeneration()) {
1861+
if (flushOrRollRunning.compareAndSet(false, true)) {
1862+
/*
1863+
* We have to check again since otherwise there is a race when a thread passes the
1864+
* first check next to another thread which performs the operation quickly enough to
1865+
* finish before the current thread could flip the flag. In that situation, we have
1866+
* an extra operation.
1867+
*
1868+
* Additionally, a flush implicitly executes a translog generation roll so if we
1869+
* execute a flush then we do not need to check if we should roll the translog
1870+
* generation.
1871+
*/
1872+
if (shouldFlush()) {
18311873
logger.debug("submitting async flush request");
1832-
final AbstractRunnable abstractRunnable = new AbstractRunnable() {
1874+
final AbstractRunnable flush = new AbstractRunnable() {
18331875
@Override
1834-
public void onFailure(Exception e) {
1876+
public void onFailure(final Exception e) {
18351877
if (state != IndexShardState.CLOSED) {
18361878
logger.warn("failed to flush index", e);
18371879
}
@@ -1844,16 +1886,38 @@ protected void doRun() throws Exception {
18441886

18451887
@Override
18461888
public void onAfter() {
1847-
asyncFlushRunning.compareAndSet(true, false);
1848-
maybeFlush(); // fire a flush up again if we have filled up the limits such that shouldFlush() returns true
1889+
flushOrRollRunning.compareAndSet(true, false);
1890+
afterWriteOperation();
18491891
}
18501892
};
1851-
threadPool.executor(ThreadPool.Names.FLUSH).execute(abstractRunnable);
1852-
return true;
1893+
threadPool.executor(ThreadPool.Names.FLUSH).execute(flush);
1894+
} else if (shouldRollTranslogGeneration()) {
1895+
logger.debug("submitting async roll translog generation request");
1896+
final AbstractRunnable roll = new AbstractRunnable() {
1897+
@Override
1898+
public void onFailure(final Exception e) {
1899+
if (state != IndexShardState.CLOSED) {
1900+
logger.warn("failed to roll translog generation", e);
1901+
}
1902+
}
1903+
1904+
@Override
1905+
protected void doRun() throws Exception {
1906+
rollTranslogGeneration();
1907+
}
1908+
1909+
@Override
1910+
public void onAfter() {
1911+
flushOrRollRunning.compareAndSet(true, false);
1912+
afterWriteOperation();
1913+
}
1914+
};
1915+
threadPool.executor(ThreadPool.Names.FLUSH).execute(roll);
1916+
} else {
1917+
flushOrRollRunning.compareAndSet(true, false);
18531918
}
18541919
}
18551920
}
1856-
return false;
18571921
}
18581922

18591923
/**

0 commit comments

Comments
 (0)