Skip to content

Commit 45c6c20

Browse files
authored
Enforce translog access via engine (#29542)
Today the translog of an engine is exposed and can be accessed directly. While this exposure offers much flexibility, it also causes these troubles: - Inconsistent behavior between translog method and engine method. For example, rolling a translog generation via an engine also trims unreferenced files, but translog's method does not. - An engine does not get notified when critical errors happen in translog as the access is direct. This change isolates translog of an engine and enforces all accesses to translog via the engine.
1 parent 1dd0fd4 commit 45c6c20

21 files changed

+170
-79
lines changed

server/src/main/java/org/elasticsearch/index/IndexService.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -697,8 +697,7 @@ private void maybeFSyncTranslogs() {
697697
if (indexSettings.getTranslogDurability() == Translog.Durability.ASYNC) {
698698
for (IndexShard shard : this.shards.values()) {
699699
try {
700-
Translog translog = shard.getTranslog();
701-
if (translog.syncNeeded()) {
700+
if (shard.isSyncNeeded()) {
702701
shard.sync();
703702
}
704703
} catch (AlreadyClosedException ex) {

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

Lines changed: 60 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,7 @@
6666
import org.elasticsearch.index.shard.ShardId;
6767
import org.elasticsearch.index.store.Store;
6868
import org.elasticsearch.index.translog.Translog;
69+
import org.elasticsearch.index.translog.TranslogStats;
6970

7071
import java.io.Closeable;
7172
import java.io.FileNotFoundException;
@@ -510,8 +511,18 @@ public enum SearcherScope {
510511
EXTERNAL, INTERNAL
511512
}
512513

513-
/** returns the translog for this engine */
514-
public abstract Translog getTranslog();
514+
/**
515+
* Returns the translog associated with this engine.
516+
* Prefer to keep the translog package-private, so that an engine can control all accesses to the translog.
517+
*/
518+
abstract Translog getTranslog();
519+
520+
/**
521+
* Checks if the underlying storage sync is required.
522+
*/
523+
public boolean isTranslogSyncNeeded() {
524+
return getTranslog().syncNeeded();
525+
}
515526

516527
/**
517528
* Ensures that all locations in the given stream have been written to the underlying storage.
@@ -520,6 +531,36 @@ public enum SearcherScope {
520531

521532
public abstract void syncTranslog() throws IOException;
522533

534+
public Closeable acquireTranslogRetentionLock() {
535+
return getTranslog().acquireRetentionLock();
536+
}
537+
538+
/**
539+
* Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#.
540+
* The caller has to close the returned snapshot after finishing the reading.
541+
*/
542+
public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException {
543+
return getTranslog().newSnapshotFromMinSeqNo(minSeqNo);
544+
}
545+
546+
/**
547+
* Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#.
548+
*/
549+
public int estimateTranslogOperationsFromMinSeq(long minSeqNo) {
550+
return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo);
551+
}
552+
553+
public TranslogStats getTranslogStats() {
554+
return getTranslog().stats();
555+
}
556+
557+
/**
558+
* Returns the last location that the translog of this engine has written into.
559+
*/
560+
public Translog.Location getTranslogLastWriteLocation() {
561+
return getTranslog().getLastWriteLocation();
562+
}
563+
523564
protected final void ensureOpen(Exception suppressed) {
524565
if (isClosed.get()) {
525566
AlreadyClosedException ace = new AlreadyClosedException(shardId + " engine is closed", failedEngine.get());
@@ -546,6 +587,13 @@ public CommitStats commitStats() {
546587
*/
547588
public abstract LocalCheckpointTracker getLocalCheckpointTracker();
548589

590+
/**
591+
* Returns the latest global checkpoint value that has been persisted in the underlying storage (i.e. translog's checkpoint)
592+
*/
593+
public long getLastSyncedGlobalCheckpoint() {
594+
return getTranslog().getLastSyncedGlobalCheckpoint();
595+
}
596+
549597
/**
550598
* Global stats on segments.
551599
*/
@@ -810,6 +858,16 @@ public final boolean refreshNeeded() {
810858
*/
811859
public abstract void trimTranslog() throws EngineException;
812860

861+
/**
862+
* Tests whether or not the translog generation should be rolled to a new generation.
863+
* This test is based on the size of the current generation compared to the configured generation threshold size.
864+
*
865+
* @return {@code true} if the current generation should be rolled to a new generation
866+
*/
867+
public boolean shouldRollTranslogGeneration() {
868+
return getTranslog().shouldRollGeneration();
869+
}
870+
813871
/**
814872
* Rolls the translog generation and cleans unneeded.
815873
*/

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -422,7 +422,7 @@ private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy
422422
}
423423

424424
@Override
425-
public Translog getTranslog() {
425+
Translog getTranslog() {
426426
ensureOpen();
427427
return translog;
428428
}

server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -131,7 +131,7 @@ protected ReplicaResult shardOperationOnReplica(final Request request, final Ind
131131

132132
private void maybeSyncTranslog(final IndexShard indexShard) throws IOException {
133133
if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST &&
134-
indexShard.getTranslog().getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) {
134+
indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) {
135135
indexShard.sync();
136136
}
137137
}

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

Lines changed: 37 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -934,7 +934,7 @@ public FieldDataStats fieldDataStats(String... fields) {
934934
}
935935

936936
public TranslogStats translogStats() {
937-
return getEngine().getTranslog().stats();
937+
return getEngine().getTranslogStats();
938938
}
939939

940940
public CompletionStats completionStats(String... fields) {
@@ -1331,7 +1331,7 @@ private boolean assertMaxUnsafeAutoIdInCommit() throws IOException {
13311331
}
13321332

13331333
protected void onNewEngine(Engine newEngine) {
1334-
refreshListeners.setTranslog(newEngine.getTranslog());
1334+
refreshListeners.setCurrentRefreshLocationSupplier(newEngine::getTranslogLastWriteLocation);
13351335
}
13361336

13371337
/**
@@ -1563,8 +1563,7 @@ boolean shouldRollTranslogGeneration() {
15631563
final Engine engine = getEngineOrNull();
15641564
if (engine != null) {
15651565
try {
1566-
final Translog translog = engine.getTranslog();
1567-
return translog.shouldRollGeneration();
1566+
return engine.shouldRollTranslogGeneration();
15681567
} catch (final AlreadyClosedException e) {
15691568
// we are already closed, no need to flush or roll
15701569
}
@@ -1579,9 +1578,26 @@ public void onSettingsChanged() {
15791578
}
15801579
}
15811580

1581+
/**
1582+
* Acquires a lock on the translog files, preventing them from being trimmed.
1583+
*/
15821584
public Closeable acquireTranslogRetentionLock() {
1583-
Engine engine = getEngine();
1584-
return engine.getTranslog().acquireRetentionLock();
1585+
return getEngine().acquireTranslogRetentionLock();
1586+
}
1587+
1588+
/**
1589+
* Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#.
1590+
* The caller has to close the returned snapshot after finishing the reading.
1591+
*/
1592+
public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException {
1593+
return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo);
1594+
}
1595+
1596+
/**
1597+
* Returns the estimated number of operations in translog whose seq# at least the provided seq#.
1598+
*/
1599+
public int estimateTranslogOperationsFromMinSeq(long minSeqNo) {
1600+
return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo);
15851601
}
15861602

15871603
public List<Segment> segments(boolean verbose) {
@@ -1592,10 +1608,6 @@ public void flushAndCloseEngine() throws IOException {
15921608
getEngine().flushAndClose();
15931609
}
15941610

1595-
public Translog getTranslog() {
1596-
return getEngine().getTranslog();
1597-
}
1598-
15991611
public String getHistoryUUID() {
16001612
return getEngine().getHistoryUUID();
16011613
}
@@ -1733,6 +1745,13 @@ public long getGlobalCheckpoint() {
17331745
return replicationTracker.getGlobalCheckpoint();
17341746
}
17351747

1748+
/**
1749+
* Returns the latest global checkpoint value that has been persisted in the underlying storage (i.e. translog's checkpoint)
1750+
*/
1751+
public long getLastSyncedGlobalCheckpoint() {
1752+
return getEngine().getLastSyncedGlobalCheckpoint();
1753+
}
1754+
17361755
/**
17371756
* Get the local knowledge of the global checkpoints for all in-sync allocation IDs.
17381757
*
@@ -2308,6 +2327,13 @@ public void sync() throws IOException {
23082327
getEngine().syncTranslog();
23092328
}
23102329

2330+
/**
2331+
* Checks if the underlying storage sync is required.
2332+
*/
2333+
public boolean isSyncNeeded() {
2334+
return getEngine().isTranslogSyncNeeded();
2335+
}
2336+
23112337
/**
23122338
* Returns the current translog durability mode
23132339
*/
@@ -2467,7 +2493,7 @@ final long getLastSearcherAccess() {
24672493
}
24682494

24692495
private void setRefreshPending(Engine engine) {
2470-
Translog.Location lastWriteLocation = engine.getTranslog().getLastWriteLocation();
2496+
Translog.Location lastWriteLocation = engine.getTranslogLastWriteLocation();
24712497
Translog.Location location;
24722498
do {
24732499
location = this.pendingRefreshLocation.get();

server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@ public void resync(final IndexShard indexShard, final ActionListener<ResyncTask>
8383
ActionListener<ResyncTask> resyncListener = null;
8484
try {
8585
final long startingSeqNo = indexShard.getGlobalCheckpoint() + 1;
86-
Translog.Snapshot snapshot = indexShard.getTranslog().newSnapshotFromMinSeqNo(startingSeqNo);
86+
Translog.Snapshot snapshot = indexShard.newTranslogSnapshotFromMinSeqNo(startingSeqNo);
8787
resyncListener = new ActionListener<ResyncTask>() {
8888
@Override
8989
public void onResponse(final ResyncTask resyncTask) {

server/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import java.util.concurrent.Executor;
3333
import java.util.function.Consumer;
3434
import java.util.function.IntSupplier;
35+
import java.util.function.Supplier;
3536

3637
import static java.util.Objects.requireNonNull;
3738

@@ -153,21 +154,20 @@ public int pendingCount() {
153154
/**
154155
* Setup the translog used to find the last refreshed location.
155156
*/
156-
public void setTranslog(Translog translog) {
157-
this.translog = translog;
157+
public void setCurrentRefreshLocationSupplier(Supplier<Translog.Location> currentRefreshLocationSupplier) {
158+
this.currentRefreshLocationSupplier = currentRefreshLocationSupplier;
158159
}
159160

160-
// Implementation of ReferenceManager.RefreshListener that adapts Lucene's RefreshListener into Elasticsearch's refresh listeners.
161-
private Translog translog;
162161
/**
163162
* Snapshot of the translog location before the current refresh if there is a refresh going on or null. Doesn't have to be volatile
164163
* because when it is used by the refreshing thread.
165164
*/
166165
private Translog.Location currentRefreshLocation;
166+
private Supplier<Translog.Location> currentRefreshLocationSupplier;
167167

168168
@Override
169169
public void beforeRefresh() throws IOException {
170-
currentRefreshLocation = translog.getLastWriteLocation();
170+
currentRefreshLocation = currentRefreshLocationSupplier.get();
171171
}
172172

173173
@Override

server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -145,9 +145,6 @@ public RecoveryResponse recoverToTarget() throws IOException {
145145
}, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ");
146146

147147
try (Closeable ignored = shard.acquireTranslogRetentionLock()) {
148-
149-
final Translog translog = shard.getTranslog();
150-
151148
final long startingSeqNo;
152149
final long requiredSeqNoRangeStart;
153150
final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO &&
@@ -170,7 +167,7 @@ public RecoveryResponse recoverToTarget() throws IOException {
170167
requiredSeqNoRangeStart =
171168
Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1;
172169
try {
173-
phase1(phase1Snapshot.getIndexCommit(), translog::totalOperations);
170+
phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo));
174171
} catch (final Exception e) {
175172
throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
176173
} finally {
@@ -187,7 +184,7 @@ public RecoveryResponse recoverToTarget() throws IOException {
187184

188185
try {
189186
// For a sequence based recovery, the target can keep its local translog
190-
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, translog.estimateTotalOperationsFromMinSeq(startingSeqNo));
187+
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo));
191188
} catch (final Exception e) {
192189
throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e);
193190
}
@@ -210,9 +207,9 @@ public RecoveryResponse recoverToTarget() throws IOException {
210207

211208
logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo);
212209

213-
logger.trace("snapshot translog for recovery; current size is [{}]", translog.estimateTotalOperationsFromMinSeq(startingSeqNo));
210+
logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo));
214211
final long targetLocalCheckpoint;
215-
try(Translog.Snapshot snapshot = translog.newSnapshotFromMinSeqNo(startingSeqNo)) {
212+
try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) {
216213
targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot);
217214
} catch (Exception e) {
218215
throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e);
@@ -261,7 +258,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException {
261258
// the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one
262259
if (startingSeqNo - 1 <= localCheckpoint) {
263260
final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1);
264-
try (Translog.Snapshot snapshot = shard.getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) {
261+
try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) {
265262
Translog.Operation operation;
266263
while ((operation = snapshot.next()) != null) {
267264
if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) {

server/src/test/java/org/elasticsearch/index/IndexServiceTests.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -250,7 +250,7 @@ public void testAsyncFsyncActuallyWorks() throws Exception {
250250
client().prepareIndex("test", "test", "1").setSource("{\"foo\": \"bar\"}", XContentType.JSON).get();
251251
IndexShard shard = indexService.getShard(0);
252252
assertBusy(() -> {
253-
assertFalse(shard.getTranslog().syncNeeded());
253+
assertFalse(shard.isSyncNeeded());
254254
});
255255
}
256256

@@ -275,7 +275,7 @@ public void testRescheduleAsyncFsync() throws Exception {
275275
client().prepareIndex("test", "test", "1").setSource("{\"foo\": \"bar\"}", XContentType.JSON).get();
276276
assertNotNull(indexService.getFsyncTask());
277277
final IndexShard shard = indexService.getShard(0);
278-
assertBusy(() -> assertFalse(shard.getTranslog().syncNeeded()));
278+
assertBusy(() -> assertFalse(shard.isSyncNeeded()));
279279

280280
client()
281281
.admin()
@@ -311,7 +311,7 @@ public void testAsyncTranslogTrimActuallyWorks() throws Exception {
311311
indexService.updateMetaData(metaData);
312312

313313
IndexShard shard = indexService.getShard(0);
314-
assertBusy(() -> assertThat(shard.getTranslog().totalOperations(), equalTo(0)));
314+
assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)));
315315
}
316316

317317
public void testIllegalFsyncInterval() {

server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -681,7 +681,7 @@ class GlobalCheckpointSync extends ReplicationAction<
681681
@Override
682682
protected PrimaryResult performOnPrimary(
683683
final IndexShard primary, final GlobalCheckpointSyncAction.Request request) throws Exception {
684-
primary.getTranslog().sync();
684+
primary.sync();
685685
return new PrimaryResult(request, new ReplicationResponse());
686686
}
687687

server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -330,7 +330,7 @@ public void testSeqNoCollision() throws Exception {
330330

331331
final Translog.Operation op1;
332332
final List<Translog.Operation> initOperations = new ArrayList<>(initDocs);
333-
try (Translog.Snapshot snapshot = replica2.getTranslog().newSnapshot()) {
333+
try (Translog.Snapshot snapshot = getTranslog(replica2).newSnapshot()) {
334334
assertThat(snapshot.totalOperations(), equalTo(initDocs + 1));
335335
for (int i = 0; i < initDocs; i++) {
336336
Translog.Operation op = snapshot.next();
@@ -347,7 +347,7 @@ public void testSeqNoCollision() throws Exception {
347347
shards.promoteReplicaToPrimary(replica1).get(); // wait until resync completed.
348348
shards.index(new IndexRequest(index.getName(), "type", "d2").source("{}", XContentType.JSON));
349349
final Translog.Operation op2;
350-
try (Translog.Snapshot snapshot = replica2.getTranslog().newSnapshot()) {
350+
try (Translog.Snapshot snapshot = getTranslog(replica2).newSnapshot()) {
351351
assertThat(snapshot.totalOperations(), equalTo(initDocs + 2));
352352
op2 = snapshot.next();
353353
assertThat(op2.seqNo(), equalTo(op1.seqNo()));
@@ -362,7 +362,7 @@ public void testSeqNoCollision() throws Exception {
362362
shards.promoteReplicaToPrimary(replica2);
363363
logger.info("--> Recover replica3 from replica2");
364364
recoverReplica(replica3, replica2);
365-
try (Translog.Snapshot snapshot = replica3.getTranslog().newSnapshot()) {
365+
try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) {
366366
assertThat(snapshot.totalOperations(), equalTo(initDocs + 1));
367367
assertThat(snapshot.next(), equalTo(op2));
368368
assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations));
@@ -468,7 +468,7 @@ private static void assertNoOpTranslogOperationForDocumentFailure(
468468
long expectedPrimaryTerm,
469469
String failureMessage) throws IOException {
470470
for (IndexShard indexShard : replicationGroup) {
471-
try(Translog.Snapshot snapshot = indexShard.getTranslog().newSnapshot()) {
471+
try(Translog.Snapshot snapshot = getTranslog(indexShard).newSnapshot()) {
472472
assertThat(snapshot.totalOperations(), equalTo(expectedOperation));
473473
long expectedSeqNo = 0L;
474474
Translog.Operation op = snapshot.next();

0 commit comments

Comments
 (0)