Skip to content

Commit dfdf496

Browse files
authored
Move the IndexDeletionPolicy to be engine internal (#24930)
The `IndexDeletionPolicy` is currently instantiated by `IndexShard` and is then passed through to the engine as a parameter. That's a shame as it is really just an implementation detail and the engine already has a method to acquire a commit. This is preparing for a follow up PR that will we connect the index deletion policy with a new translog deletion policy. Relates to #10708
1 parent 5741005 commit dfdf496

File tree

11 files changed

+72
-89
lines changed

11 files changed

+72
-89
lines changed

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

Lines changed: 24 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import org.apache.lucene.util.Accountables;
4545
import org.apache.lucene.util.SetOnce;
4646
import org.elasticsearch.ExceptionsHelper;
47+
import org.elasticsearch.common.CheckedRunnable;
4748
import org.elasticsearch.common.Nullable;
4849
import org.elasticsearch.common.bytes.BytesReference;
4950
import org.elasticsearch.common.collect.ImmutableOpenMap;
@@ -63,7 +64,6 @@
6364
import org.elasticsearch.index.VersionType;
6465
import org.elasticsearch.index.mapper.ParseContext.Document;
6566
import org.elasticsearch.index.mapper.ParsedDocument;
66-
import org.elasticsearch.index.mapper.Uid;
6767
import org.elasticsearch.index.merge.MergeStats;
6868
import org.elasticsearch.index.seqno.SequenceNumbersService;
6969
import org.elasticsearch.index.shard.ShardId;
@@ -100,7 +100,6 @@ public abstract class Engine implements Closeable {
100100
protected final Store store;
101101
protected final AtomicBoolean isClosed = new AtomicBoolean(false);
102102
protected final EventListener eventListener;
103-
protected final SnapshotDeletionPolicy deletionPolicy;
104103
protected final ReentrantLock failEngineLock = new ReentrantLock();
105104
protected final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock();
106105
protected final ReleasableLock readLock = new ReleasableLock(rwl.readLock());
@@ -121,15 +120,13 @@ public abstract class Engine implements Closeable {
121120

122121
protected Engine(EngineConfig engineConfig) {
123122
Objects.requireNonNull(engineConfig.getStore(), "Store must be provided to the engine");
124-
Objects.requireNonNull(engineConfig.getDeletionPolicy(), "Snapshot deletion policy must be provided to the engine");
125123

126124
this.engineConfig = engineConfig;
127125
this.shardId = engineConfig.getShardId();
128126
this.store = engineConfig.getStore();
129127
this.logger = Loggers.getLogger(Engine.class, // we use the engine class directly here to make sure all subclasses have the same logger name
130128
engineConfig.getIndexSettings().getSettings(), engineConfig.getShardId());
131129
this.eventListener = engineConfig.getEventListener();
132-
this.deletionPolicy = engineConfig.getDeletionPolicy();
133130
}
134131

135132
/** Returns 0 in the case where accountable is null, otherwise returns {@code ramBytesUsed()} */
@@ -828,7 +825,7 @@ public void forceMerge(boolean flush) throws IOException {
828825
*
829826
* @param flushFirst indicates whether the engine should flush before returning the snapshot
830827
*/
831-
public abstract IndexCommit acquireIndexCommit(boolean flushFirst) throws EngineException;
828+
public abstract IndexCommitRef acquireIndexCommit(boolean flushFirst) throws EngineException;
832829

833830
/**
834831
* fail engine due to some error. the engine will also be closed.
@@ -1387,6 +1384,28 @@ public int hashCode() {
13871384
}
13881385
}
13891386

1387+
public static class IndexCommitRef implements Closeable {
1388+
private final AtomicBoolean closed = new AtomicBoolean();
1389+
private final CheckedRunnable<IOException> onClose;
1390+
private final IndexCommit indexCommit;
1391+
1392+
IndexCommitRef(SnapshotDeletionPolicy deletionPolicy) throws IOException {
1393+
indexCommit = deletionPolicy.snapshot();
1394+
onClose = () -> deletionPolicy.release(indexCommit);
1395+
}
1396+
1397+
@Override
1398+
public void close() throws IOException {
1399+
if (closed.compareAndSet(false, true)) {
1400+
onClose.run();
1401+
}
1402+
}
1403+
1404+
public IndexCommit getIndexCommit() {
1405+
return indexCommit;
1406+
}
1407+
}
1408+
13901409
public void onSettingsChanged() {
13911410
}
13921411

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

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
import org.apache.lucene.analysis.Analyzer;
2222
import org.apache.lucene.codecs.Codec;
2323
import org.apache.lucene.index.MergePolicy;
24-
import org.apache.lucene.index.SnapshotDeletionPolicy;
2524
import org.apache.lucene.search.QueryCache;
2625
import org.apache.lucene.search.QueryCachingPolicy;
2726
import org.apache.lucene.search.ReferenceManager;
@@ -58,7 +57,6 @@ public final class EngineConfig {
5857
private final ThreadPool threadPool;
5958
private final Engine.Warmer warmer;
6059
private final Store store;
61-
private final SnapshotDeletionPolicy deletionPolicy;
6260
private final MergePolicy mergePolicy;
6361
private final Analyzer analyzer;
6462
private final Similarity similarity;
@@ -109,7 +107,7 @@ public final class EngineConfig {
109107
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
110108
*/
111109
public EngineConfig(OpenMode openMode, ShardId shardId, ThreadPool threadPool,
112-
IndexSettings indexSettings, Engine.Warmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
110+
IndexSettings indexSettings, Engine.Warmer warmer, Store store,
113111
MergePolicy mergePolicy, Analyzer analyzer,
114112
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
115113
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
@@ -123,7 +121,6 @@ public EngineConfig(OpenMode openMode, ShardId shardId, ThreadPool threadPool,
123121
this.threadPool = threadPool;
124122
this.warmer = warmer == null ? (a) -> {} : warmer;
125123
this.store = store;
126-
this.deletionPolicy = deletionPolicy;
127124
this.mergePolicy = mergePolicy;
128125
this.analyzer = analyzer;
129126
this.similarity = similarity;
@@ -214,14 +211,6 @@ public Store getStore() {
214211
return store;
215212
}
216213

217-
/**
218-
* Returns a {@link SnapshotDeletionPolicy} used in the engines
219-
* {@link org.apache.lucene.index.IndexWriter}.
220-
*/
221-
public SnapshotDeletionPolicy getDeletionPolicy() {
222-
return deletionPolicy;
223-
}
224-
225214
/**
226215
* Returns the {@link org.apache.lucene.index.MergePolicy} for the engines {@link org.apache.lucene.index.IndexWriter}
227216
*/

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

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,16 +21,17 @@
2121

2222
import org.apache.logging.log4j.Logger;
2323
import org.apache.lucene.index.DirectoryReader;
24-
import org.apache.lucene.index.IndexCommit;
2524
import org.apache.lucene.index.IndexFormatTooOldException;
2625
import org.apache.lucene.index.IndexReader;
2726
import org.apache.lucene.index.IndexWriter;
2827
import org.apache.lucene.index.IndexWriterConfig;
28+
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
2929
import org.apache.lucene.index.LeafReader;
3030
import org.apache.lucene.index.LiveIndexWriterConfig;
3131
import org.apache.lucene.index.MergePolicy;
3232
import org.apache.lucene.index.SegmentCommitInfo;
3333
import org.apache.lucene.index.SegmentInfos;
34+
import org.apache.lucene.index.SnapshotDeletionPolicy;
3435
import org.apache.lucene.index.Term;
3536
import org.apache.lucene.search.IndexSearcher;
3637
import org.apache.lucene.search.SearcherFactory;
@@ -126,6 +127,8 @@ public class InternalEngine extends Engine {
126127

127128
private final String uidField;
128129

130+
private final SnapshotDeletionPolicy deletionPolicy;
131+
129132
// How many callers are currently requesting index throttling. Currently there are only two situations where we do this: when merges
130133
// are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling
131134
// incoming indexing ops to a single thread:
@@ -137,12 +140,14 @@ public class InternalEngine extends Engine {
137140
private final CounterMetric numVersionLookups = new CounterMetric();
138141
private final CounterMetric numIndexVersionsLookups = new CounterMetric();
139142

143+
140144
public InternalEngine(EngineConfig engineConfig) throws EngineException {
141145
super(engineConfig);
142146
openMode = engineConfig.getOpenMode();
143147
if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) {
144148
maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE);
145149
}
150+
deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
146151
this.uidField = engineConfig.getIndexSettings().isSingleType() ? IdFieldMapper.NAME : UidFieldMapper.NAME;
147152
this.versionMap = new LiveVersionMap();
148153
store.incRef();
@@ -1414,7 +1419,7 @@ public void forceMerge(final boolean flush, int maxNumSegments, boolean onlyExpu
14141419
}
14151420

14161421
@Override
1417-
public IndexCommit acquireIndexCommit(final boolean flushFirst) throws EngineException {
1422+
public IndexCommitRef acquireIndexCommit(final boolean flushFirst) throws EngineException {
14181423
// we have to flush outside of the readlock otherwise we might have a problem upgrading
14191424
// the to a write lock when we fail the engine in this operation
14201425
if (flushFirst) {
@@ -1425,7 +1430,7 @@ public IndexCommit acquireIndexCommit(final boolean flushFirst) throws EngineExc
14251430
try (ReleasableLock lock = readLock.acquire()) {
14261431
ensureOpen();
14271432
logger.trace("pulling snapshot");
1428-
return deletionPolicy.snapshot();
1433+
return new IndexCommitRef(deletionPolicy);
14291434
} catch (IOException e) {
14301435
throw new SnapshotFailedEngineException(shardId, e);
14311436
}

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

Lines changed: 9 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,7 @@
2323
import org.apache.lucene.index.CheckIndex;
2424
import org.apache.lucene.index.IndexCommit;
2525
import org.apache.lucene.index.IndexOptions;
26-
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
2726
import org.apache.lucene.index.SegmentInfos;
28-
import org.apache.lucene.index.SnapshotDeletionPolicy;
2927
import org.apache.lucene.index.Term;
3028
import org.apache.lucene.search.QueryCachingPolicy;
3129
import org.apache.lucene.search.Sort;
@@ -161,7 +159,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
161159
private final String checkIndexOnStartup;
162160
private final CodecService codecService;
163161
private final Engine.Warmer warmer;
164-
private final SnapshotDeletionPolicy deletionPolicy;
165162
private final SimilarityService similarityService;
166163
private final TranslogConfig translogConfig;
167164
private final IndexEventListener indexEventListener;
@@ -230,7 +227,6 @@ public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardP
230227
final Settings settings = indexSettings.getSettings();
231228
this.codecService = new CodecService(mapperService, logger);
232229
this.warmer = warmer;
233-
this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
234230
this.similarityService = similarityService;
235231
Objects.requireNonNull(store, "Store must be provided to the index shard");
236232
this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
@@ -878,11 +874,11 @@ public org.apache.lucene.util.Version minimumCompatibleVersion() {
878874

879875
/**
880876
* Creates a new {@link IndexCommit} snapshot form the currently running engine. All resources referenced by this
881-
* commit won't be freed until the commit / snapshot is released via {@link #releaseIndexCommit(IndexCommit)}.
877+
* commit won't be freed until the commit / snapshot is closed.
882878
*
883879
* @param flushFirst <code>true</code> if the index should first be flushed to disk / a low level lucene commit should be executed
884880
*/
885-
public IndexCommit acquireIndexCommit(boolean flushFirst) throws EngineException {
881+
public Engine.IndexCommitRef acquireIndexCommit(boolean flushFirst) throws EngineException {
886882
IndexShardState state = this.state; // one time volatile read
887883
// we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine
888884
if (state == IndexShardState.STARTED || state == IndexShardState.RELOCATED || state == IndexShardState.CLOSED) {
@@ -893,14 +889,6 @@ public IndexCommit acquireIndexCommit(boolean flushFirst) throws EngineException
893889
}
894890

895891

896-
/**
897-
* Releases a snapshot taken from {@link #acquireIndexCommit(boolean)} this must be called to release the resources
898-
* referenced by the given snapshot {@link IndexCommit}.
899-
*/
900-
public void releaseIndexCommit(IndexCommit snapshot) throws IOException {
901-
deletionPolicy.release(snapshot);
902-
}
903-
904892
/**
905893
* gets a {@link Store.MetadataSnapshot} for the current directory. This method is safe to call in all lifecycle of the index shard,
906894
* without having to worry about the current state of the engine and concurrent flushes.
@@ -915,25 +903,24 @@ public void releaseIndexCommit(IndexCommit snapshot) throws IOException {
915903
* @throws java.nio.file.NoSuchFileException if one or more files referenced by a commit are not present.
916904
*/
917905
public Store.MetadataSnapshot snapshotStoreMetadata() throws IOException {
918-
IndexCommit indexCommit = null;
906+
Engine.IndexCommitRef indexCommit = null;
919907
store.incRef();
920908
try {
909+
Engine engine;
921910
synchronized (mutex) {
922911
// if the engine is not running, we can access the store directly, but we need to make sure no one starts
923912
// the engine on us. If the engine is running, we can get a snapshot via the deletion policy which is initialized.
924913
// That can be done out of mutex, since the engine can be closed half way.
925-
Engine engine = getEngineOrNull();
914+
engine = getEngineOrNull();
926915
if (engine == null) {
927916
return store.getMetadata(null, true);
928917
}
929918
}
930-
indexCommit = deletionPolicy.snapshot();
931-
return store.getMetadata(indexCommit);
919+
indexCommit = engine.acquireIndexCommit(false);
920+
return store.getMetadata(indexCommit.getIndexCommit());
932921
} finally {
933922
store.decRef();
934-
if (indexCommit != null) {
935-
deletionPolicy.release(indexCommit);
936-
}
923+
IOUtils.close(indexCommit);
937924
}
938925
}
939926

@@ -1838,7 +1825,7 @@ private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode) {
18381825
final IndexShardRecoveryPerformer translogRecoveryPerformer = new IndexShardRecoveryPerformer(shardId, mapperService, logger);
18391826
Sort indexSort = indexSortSupplier.get();
18401827
return new EngineConfig(openMode, shardId,
1841-
threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(),
1828+
threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(),
18421829
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
18431830
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners, indexSort);
18441831
}

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

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919

2020
package org.elasticsearch.index.shard;
2121

22-
import org.apache.lucene.index.IndexCommit;
2322
import org.apache.lucene.store.Directory;
2423
import org.apache.lucene.store.FilterDirectory;
2524
import org.apache.lucene.store.IOContext;
@@ -28,6 +27,7 @@
2827
import org.apache.lucene.store.NoLockFactory;
2928
import org.elasticsearch.cluster.metadata.IndexMetaData;
3029
import org.elasticsearch.index.Index;
30+
import org.elasticsearch.index.engine.Engine;
3131
import org.elasticsearch.index.store.Store;
3232

3333
import java.io.Closeable;
@@ -38,7 +38,7 @@
3838
final class LocalShardSnapshot implements Closeable {
3939
private final IndexShard shard;
4040
private final Store store;
41-
private final IndexCommit indexCommit;
41+
private final Engine.IndexCommitRef indexCommit;
4242
private final AtomicBoolean closed = new AtomicBoolean(false);
4343

4444
LocalShardSnapshot(IndexShard shard) {
@@ -66,7 +66,7 @@ Directory getSnapshotDirectory() {
6666
return new FilterDirectory(store.directory()) {
6767
@Override
6868
public String[] listAll() throws IOException {
69-
Collection<String> fileNames = indexCommit.getFileNames();
69+
Collection<String> fileNames = indexCommit.getIndexCommit().getFileNames();
7070
final String[] fileNameArray = fileNames.toArray(new String[fileNames.size()]);
7171
return fileNameArray;
7272
}
@@ -115,7 +115,7 @@ public void close() throws IOException {
115115
public void close() throws IOException {
116116
if (closed.compareAndSet(false, true)) {
117117
try {
118-
shard.releaseIndexCommit(indexCommit);
118+
indexCommit.close();
119119
} finally {
120120
store.decRef();
121121
}

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import org.elasticsearch.common.settings.Settings;
4242
import org.elasticsearch.common.unit.ByteSizeValue;
4343
import org.elasticsearch.common.util.CancellableThreads;
44+
import org.elasticsearch.index.engine.Engine;
4445
import org.elasticsearch.index.engine.RecoveryEngineException;
4546
import org.elasticsearch.index.seqno.LocalCheckpointTracker;
4647
import org.elasticsearch.index.seqno.SequenceNumbersService;
@@ -135,20 +136,20 @@ public RecoveryResponse recoverToTarget() throws IOException {
135136
if (isSequenceNumberBasedRecoveryPossible) {
136137
logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo());
137138
} else {
138-
final IndexCommit phase1Snapshot;
139+
final Engine.IndexCommitRef phase1Snapshot;
139140
try {
140141
phase1Snapshot = shard.acquireIndexCommit(false);
141142
} catch (final Exception e) {
142143
IOUtils.closeWhileHandlingException(translogView);
143144
throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e);
144145
}
145146
try {
146-
phase1(phase1Snapshot, translogView);
147+
phase1(phase1Snapshot.getIndexCommit(), translogView);
147148
} catch (final Exception e) {
148149
throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
149150
} finally {
150151
try {
151-
shard.releaseIndexCommit(phase1Snapshot);
152+
IOUtils.close(phase1Snapshot);
152153
} catch (final IOException ex) {
153154
logger.warn("releasing snapshot caused exception", ex);
154155
}

core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
2323
import org.apache.logging.log4j.message.ParameterizedMessage;
2424
import org.apache.logging.log4j.util.Supplier;
25-
import org.apache.lucene.index.IndexCommit;
2625
import org.elasticsearch.ExceptionsHelper;
2726
import org.elasticsearch.cluster.ClusterChangedEvent;
2827
import org.elasticsearch.cluster.ClusterState;
@@ -45,6 +44,7 @@
4544
import org.elasticsearch.common.unit.ByteSizeValue;
4645
import org.elasticsearch.common.unit.TimeValue;
4746
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
47+
import org.elasticsearch.index.engine.Engine;
4848
import org.elasticsearch.index.engine.SnapshotFailedEngineException;
4949
import org.elasticsearch.index.shard.IndexEventListener;
5050
import org.elasticsearch.index.shard.IndexShard;
@@ -376,17 +376,14 @@ private void snapshot(final IndexShard indexShard, final Snapshot snapshot, fina
376376

377377
try {
378378
// we flush first to make sure we get the latest writes snapshotted
379-
IndexCommit snapshotIndexCommit = indexShard.acquireIndexCommit(true);
380-
try {
381-
repository.snapshotShard(indexShard, snapshot.getSnapshotId(), indexId, snapshotIndexCommit, snapshotStatus);
379+
try (Engine.IndexCommitRef snapshotRef = indexShard.acquireIndexCommit(true)) {
380+
repository.snapshotShard(indexShard, snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), snapshotStatus);
382381
if (logger.isDebugEnabled()) {
383382
StringBuilder sb = new StringBuilder();
384383
sb.append(" index : version [").append(snapshotStatus.indexVersion()).append("], number_of_files [").append(snapshotStatus.numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(snapshotStatus.totalSize())).append("]\n");
385384
logger.debug("snapshot ({}) completed to {}, took [{}]\n{}", snapshot, repository,
386385
TimeValue.timeValueMillis(snapshotStatus.time()), sb);
387386
}
388-
} finally {
389-
indexShard.releaseIndexCommit(snapshotIndexCommit);
390387
}
391388
} catch (SnapshotFailedEngineException e) {
392389
throw e;

0 commit comments

Comments
 (0)