Add EngineConfig extensions to EnginePlugin (#1387)

This commit adds an extension point to EngineConfig through EnginePlugin using
a new EngineConfigFactory mechanism. EnginePlugin provides interface methods to
override configurations in EngineConfig. The EngineConfigFactory produces a new
instance of the EngineConfig using these overrides. Defaults are used absent
overridden configurations.

This serves as a mechanism to override Engine configurations (e.g., CodecService,
TranslogConfig) enabling Plugins to have higher fidelity for changing Engine
behavior without having to override the entire Engine (which is only permitted for
a single plugin).

Signed-off-by: Nicholas Walter Knize <nknize@apache.org>
This commit is contained in:
Nick Knize 2021-10-19 23:04:28 -05:00 committed by GitHub
parent 2da858ccb0
commit ecac8d3c38
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 208 additions and 9 deletions

View File

@ -768,6 +768,7 @@ public class IndexShardIT extends OpenSearchSingleNodeTestCase {
indexService.mapperService(),
indexService.similarityService(),
shard.getEngineFactory(),
shard.getEngineConfigFactory(),
indexService.getIndexEventListener(),
wrapper,
indexService.getThreadPool(),

View File

@ -62,6 +62,7 @@ import org.opensearch.index.cache.query.DisabledQueryCache;
import org.opensearch.index.cache.query.IndexQueryCache;
import org.opensearch.index.cache.query.QueryCache;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.mapper.MapperService;
import org.opensearch.index.shard.IndexEventListener;
@ -165,6 +166,7 @@ public final class IndexModule {
private final IndexSettings indexSettings;
private final AnalysisRegistry analysisRegistry;
private final EngineFactory engineFactory;
private final EngineConfigFactory engineConfigFactory;
private SetOnce<Function<IndexService, CheckedFunction<DirectoryReader, DirectoryReader, IOException>>> indexReaderWrapper =
new SetOnce<>();
private final Set<IndexEventListener> indexEventListeners = new HashSet<>();
@ -191,6 +193,7 @@ public final class IndexModule {
final IndexSettings indexSettings,
final AnalysisRegistry analysisRegistry,
final EngineFactory engineFactory,
final EngineConfigFactory engineConfigFactory,
final Map<String, IndexStorePlugin.DirectoryFactory> directoryFactories,
final BooleanSupplier allowExpensiveQueries,
final IndexNameExpressionResolver expressionResolver,
@ -199,6 +202,7 @@ public final class IndexModule {
this.indexSettings = indexSettings;
this.analysisRegistry = analysisRegistry;
this.engineFactory = Objects.requireNonNull(engineFactory);
this.engineConfigFactory = Objects.requireNonNull(engineConfigFactory);
this.searchOperationListeners.add(new SearchSlowLog(indexSettings));
this.indexOperationListeners.add(new IndexingSlowLog(indexSettings));
this.directoryFactories = Collections.unmodifiableMap(directoryFactories);
@ -496,6 +500,7 @@ public final class IndexModule {
shardStoreDeleter,
indexAnalyzers,
engineFactory,
engineConfigFactory,
circuitBreakerService,
bigArrays,
threadPool,

View File

@ -70,6 +70,7 @@ import org.opensearch.index.cache.IndexCache;
import org.opensearch.index.cache.bitset.BitsetFilterCache;
import org.opensearch.index.cache.query.QueryCache;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.fielddata.IndexFieldDataCache;
import org.opensearch.index.fielddata.IndexFieldDataService;
@ -138,6 +139,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
private final NamedWriteableRegistry namedWriteableRegistry;
private final SimilarityService similarityService;
private final EngineFactory engineFactory;
private final EngineConfigFactory engineConfigFactory;
private final IndexWarmer warmer;
private volatile Map<Integer, IndexShard> shards = emptyMap();
private final AtomicBoolean closed = new AtomicBoolean(false);
@ -174,6 +176,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
ShardStoreDeleter shardStoreDeleter,
IndexAnalyzers indexAnalyzers,
EngineFactory engineFactory,
EngineConfigFactory engineConfigFactory,
CircuitBreakerService circuitBreakerService,
BigArrays bigArrays,
ThreadPool threadPool,
@ -254,6 +257,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
this.directoryFactory = directoryFactory;
this.recoveryStateFactory = recoveryStateFactory;
this.engineFactory = Objects.requireNonNull(engineFactory);
this.engineConfigFactory = Objects.requireNonNull(engineConfigFactory);
// initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE
this.readerWrapper = wrapperFactory.apply(this);
this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners);
@ -507,6 +511,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
mapperService,
similarityService,
engineFactory,
engineConfigFactory,
eventListener,
readerWrapper,
threadPool,

View File

@ -0,0 +1,127 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index.engine;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.similarities.Similarity;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.codec.CodecService;
import org.opensearch.index.seqno.RetentionLeases;
import org.opensearch.index.shard.ShardId;
import org.opensearch.index.store.Store;
import org.opensearch.index.translog.TranslogConfig;
import org.opensearch.indices.breaker.CircuitBreakerService;
import org.opensearch.plugins.EnginePlugin;
import org.opensearch.plugins.PluginsService;
import org.opensearch.threadpool.ThreadPool;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
/**
* A factory to create an EngineConfig based on custom plugin overrides
*/
public class EngineConfigFactory {
private final Optional<CodecService> codecService;
/** default ctor primarily used for tests without plugins */
public EngineConfigFactory(IndexSettings idxSettings) {
this(Collections.emptyList(), idxSettings);
}
/**
* Construct a factory using the plugin service and provided index settings
*/
public EngineConfigFactory(PluginsService pluginsService, IndexSettings idxSettings) {
this(pluginsService.filterPlugins(EnginePlugin.class), idxSettings);
}
/* private constructor to construct the factory from specific EnginePlugins and IndexSettings */
EngineConfigFactory(Collection<EnginePlugin> enginePlugins, IndexSettings idxSettings) {
Optional<CodecService> codecService = Optional.empty();
String codecServiceOverridingPlugin = null;
for (EnginePlugin enginePlugin : enginePlugins) {
// get overriding codec service from EnginePlugin
if (codecService.isPresent() == false) {
codecService = enginePlugin.getCustomCodecService(idxSettings);
codecServiceOverridingPlugin = enginePlugin.getClass().getName();
} else {
throw new IllegalStateException(
"existing codec service already overridden in: "
+ codecServiceOverridingPlugin
+ " attempting to override again by: "
+ enginePlugin.getClass().getName()
);
}
}
this.codecService = codecService;
}
/** Insantiates a new EngineConfig from the provided custom overrides */
public EngineConfig newEngineConfig(
ShardId shardId,
ThreadPool threadPool,
IndexSettings indexSettings,
Engine.Warmer warmer,
Store store,
MergePolicy mergePolicy,
Analyzer analyzer,
Similarity similarity,
CodecService codecService,
Engine.EventListener eventListener,
QueryCache queryCache,
QueryCachingPolicy queryCachingPolicy,
TranslogConfig translogConfig,
TimeValue flushMergesAfter,
List<ReferenceManager.RefreshListener> externalRefreshListener,
List<ReferenceManager.RefreshListener> internalRefreshListener,
Sort indexSort,
CircuitBreakerService circuitBreakerService,
LongSupplier globalCheckpointSupplier,
Supplier<RetentionLeases> retentionLeasesSupplier,
LongSupplier primaryTermSupplier,
EngineConfig.TombstoneDocSupplier tombstoneDocSupplier
) {
return new EngineConfig(
shardId,
threadPool,
indexSettings,
warmer,
store,
mergePolicy,
analyzer,
similarity,
this.codecService.isPresent() == true ? this.codecService.get() : codecService,
eventListener,
queryCache,
queryCachingPolicy,
translogConfig,
flushMergesAfter,
externalRefreshListener,
internalRefreshListener,
indexSort,
circuitBreakerService,
globalCheckpointSupplier,
retentionLeasesSupplier,
primaryTermSupplier,
tombstoneDocSupplier
);
}
}

View File

@ -105,6 +105,7 @@ import org.opensearch.index.engine.CommitStats;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.Engine.GetResult;
import org.opensearch.index.engine.EngineConfig;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineException;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.engine.ReadOnlyEngine;
@ -238,6 +239,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
private final Object engineMutex = new Object(); // lock ordering: engineMutex -> mutex
private final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
final EngineFactory engineFactory;
final EngineConfigFactory engineConfigFactory;
private final IndexingOperationListener indexingOperationListeners;
private final Runnable globalCheckpointSyncer;
@ -302,7 +304,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
final IndexCache indexCache,
final MapperService mapperService,
final SimilarityService similarityService,
final @Nullable EngineFactory engineFactory,
final EngineFactory engineFactory,
final EngineConfigFactory engineConfigFactory,
final IndexEventListener indexEventListener,
final CheckedFunction<DirectoryReader, DirectoryReader, IOException> indexReaderWrapper,
final ThreadPool threadPool,
@ -323,6 +326,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
this.similarityService = similarityService;
Objects.requireNonNull(store, "Store must be provided to the index shard");
this.engineFactory = Objects.requireNonNull(engineFactory);
this.engineConfigFactory = Objects.requireNonNull(engineConfigFactory);
this.store = store;
this.indexSortSupplier = indexSortSupplier;
this.indexEventListener = indexEventListener;
@ -3179,7 +3183,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
this.warmer.warm(reader);
}
};
return new EngineConfig(
return this.engineConfigFactory.newEngineConfig(
shardId,
threadPool,
indexSettings,
@ -3705,6 +3709,10 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
return engineFactory;
}
EngineConfigFactory getEngineConfigFactory() {
return engineConfigFactory;
}
// for tests
ReplicationTracker getReplicationTracker() {
return replicationTracker;

View File

@ -106,6 +106,7 @@ import org.opensearch.index.analysis.AnalysisRegistry;
import org.opensearch.index.cache.request.ShardRequestCache;
import org.opensearch.index.engine.CommitStats;
import org.opensearch.index.engine.EngineConfig;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.engine.InternalEngineFactory;
import org.opensearch.index.engine.NoOpEngine;
@ -707,6 +708,7 @@ public class IndicesService extends AbstractLifecycleComponent
idxSettings,
analysisRegistry,
getEngineFactory(idxSettings),
getEngineConfigFactory(idxSettings),
directoryFactories,
() -> allowExpensiveQueries,
indexNameExpressionResolver,
@ -739,6 +741,10 @@ public class IndicesService extends AbstractLifecycleComponent
);
}
private EngineConfigFactory getEngineConfigFactory(final IndexSettings idxSettings) {
return new EngineConfigFactory(this.pluginsService, idxSettings);
}
private EngineFactory getEngineFactory(final IndexSettings idxSettings) {
final IndexMetadata indexMetadata = idxSettings.getIndexMetadata();
if (indexMetadata != null && indexMetadata.getState() == IndexMetadata.State.CLOSE) {
@ -781,6 +787,7 @@ public class IndicesService extends AbstractLifecycleComponent
idxSettings,
analysisRegistry,
getEngineFactory(idxSettings),
getEngineConfigFactory(idxSettings),
directoryFactories,
() -> allowExpensiveQueries,
indexNameExpressionResolver,

View File

@ -33,6 +33,7 @@
package org.opensearch.plugins;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.codec.CodecService;
import org.opensearch.index.engine.EngineFactory;
import java.util.Optional;
@ -52,4 +53,14 @@ public interface EnginePlugin {
*/
Optional<EngineFactory> getEngineFactory(IndexSettings indexSettings);
/**
* EXPERT:
* When an index is created this method is invoked for each engine plugin. Engine plugins can inspect the index settings
* to determine if a custom {@link CodecService} should be provided for the given index. A plugin that is not overriding
* the {@link CodecService} through the plugin can ignore this method and the Codec specified in the {@link IndexSettings}
* will be used.
*/
default Optional<CodecService> getCustomCodecService(IndexSettings indexSettings) {
return Optional.empty();
}
}

View File

@ -74,6 +74,7 @@ import org.opensearch.index.cache.query.DisabledQueryCache;
import org.opensearch.index.cache.query.IndexQueryCache;
import org.opensearch.index.cache.query.QueryCache;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.InternalEngineFactory;
import org.opensearch.index.engine.InternalEngineTests;
import org.opensearch.index.fielddata.IndexFieldDataCache;
@ -218,6 +219,7 @@ public class IndexModuleTests extends OpenSearchTestCase {
indexSettings,
emptyAnalysisRegistry,
engineFactory,
new EngineConfigFactory(indexSettings),
Collections.emptyMap(),
() -> true,
new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)),
@ -243,6 +245,7 @@ public class IndexModuleTests extends OpenSearchTestCase {
indexSettings,
emptyAnalysisRegistry,
new InternalEngineFactory(),
new EngineConfigFactory(indexSettings),
indexStoreFactories,
() -> true,
new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)),
@ -570,6 +573,7 @@ public class IndexModuleTests extends OpenSearchTestCase {
indexSettings,
emptyAnalysisRegistry,
new InternalEngineFactory(),
new EngineConfigFactory(indexSettings),
Collections.emptyMap(),
() -> true,
new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)),
@ -601,6 +605,7 @@ public class IndexModuleTests extends OpenSearchTestCase {
indexSettings,
emptyAnalysisRegistry,
new InternalEngineFactory(),
new EngineConfigFactory(indexSettings),
Collections.emptyMap(),
() -> true,
new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)),

View File

@ -57,7 +57,8 @@ public class NoOpEngineRecoveryTests extends IndexShardTestCase {
indexShard,
initWithSameId(shardRouting, ExistingStoreRecoverySource.INSTANCE),
indexShard.indexSettings().getIndexMetadata(),
NoOpEngine::new
NoOpEngine::new,
new EngineConfigFactory(indexShard.indexSettings())
);
recoverShardFromStore(primary);
assertEquals(primary.seqNoStats().getMaxSeqNo(), primary.getMaxSeqNoOfUpdatesOrDeletes());

View File

@ -98,6 +98,7 @@ import org.opensearch.index.engine.DocIdSeqNoAndSource;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.Engine.DeleteResult;
import org.opensearch.index.engine.EngineConfig;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineTestCase;
import org.opensearch.index.engine.InternalEngine;
import org.opensearch.index.engine.InternalEngineFactory;
@ -1695,6 +1696,7 @@ public class IndexShardTests extends IndexShardTestCase {
i -> store,
null,
new InternalEngineFactory(),
new EngineConfigFactory(new IndexSettings(metadata, metadata.getSettings())),
() -> {},
RetentionLeaseSyncer.EMPTY,
EMPTY_EVENT_LISTENER
@ -2558,6 +2560,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
shard.getEngineFactory(),
shard.getEngineConfigFactory(),
shard.getGlobalCheckpointSyncer(),
shard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER
@ -2694,6 +2697,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
wrapper,
new InternalEngineFactory(),
shard.getEngineConfigFactory(),
() -> {},
RetentionLeaseSyncer.EMPTY,
EMPTY_EVENT_LISTENER
@ -2845,6 +2849,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
wrapper,
new InternalEngineFactory(),
shard.getEngineConfigFactory(),
() -> {},
RetentionLeaseSyncer.EMPTY,
EMPTY_EVENT_LISTENER
@ -3506,6 +3511,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
indexShard.engineFactory,
indexShard.engineConfigFactory,
indexShard.getGlobalCheckpointSyncer(),
indexShard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER
@ -3562,6 +3568,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
indexShard.engineFactory,
indexShard.engineConfigFactory,
indexShard.getGlobalCheckpointSyncer(),
indexShard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER
@ -3595,6 +3602,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
indexShard.engineFactory,
indexShard.engineConfigFactory,
indexShard.getGlobalCheckpointSyncer(),
indexShard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER
@ -3648,6 +3656,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
indexShard.engineFactory,
indexShard.engineConfigFactory,
indexShard.getGlobalCheckpointSyncer(),
indexShard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER
@ -4120,6 +4129,7 @@ public class IndexShardTests extends IndexShardTestCase {
null,
null,
new InternalEngineFactory(),
new EngineConfigFactory(new IndexSettings(metadata, settings)),
() -> {},
RetentionLeaseSyncer.EMPTY,
new IndexEventListener() {
@ -4692,7 +4702,8 @@ public class IndexShardTests extends IndexShardTestCase {
protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(SeqNoStats seqNoStats) {
// just like a following shard, we need to skip this check for now.
}
}
},
shard.getEngineConfigFactory()
);
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
readonlyShard.markAsRecovering("store", new RecoveryState(readonlyShard.routingEntry(), localNode, null));

View File

@ -58,6 +58,7 @@ import org.opensearch.env.TestEnvironment;
import org.opensearch.gateway.PersistedClusterStateService;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.MergePolicyConfig;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineException;
import org.opensearch.index.engine.InternalEngineFactory;
import org.opensearch.index.seqno.RetentionLeaseSyncer;
@ -170,6 +171,7 @@ public class RemoveCorruptedShardDataCommandTests extends IndexShardTestCase {
null,
null,
new InternalEngineFactory(),
new EngineConfigFactory(new IndexSettings(indexMetadata, settings)),
() -> {},
RetentionLeaseSyncer.EMPTY,
EMPTY_EVENT_LISTENER
@ -545,6 +547,7 @@ public class RemoveCorruptedShardDataCommandTests extends IndexShardTestCase {
storeProvider,
null,
indexShard.engineFactory,
indexShard.getEngineConfigFactory(),
indexShard.getGlobalCheckpointSyncer(),
indexShard.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER

View File

@ -49,6 +49,7 @@ import org.opensearch.common.bytes.BytesArray;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.NoOpEngine;
import org.opensearch.index.mapper.SourceToParse;
import org.opensearch.index.seqno.SeqNoStats;
@ -315,7 +316,8 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
shard,
ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE),
indexMetadata,
NoOpEngine::new
NoOpEngine::new,
new EngineConfigFactory(shard.indexSettings())
);
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
replica.prepareForIndexRecovery();

View File

@ -48,6 +48,7 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.env.Environment;
import org.opensearch.env.TestEnvironment;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.InternalEngineFactory;
import org.opensearch.index.seqno.RetentionLeaseSyncer;
import org.opensearch.index.shard.IndexShard;
@ -136,6 +137,7 @@ public class BlobStoreRepositoryRestoreTests extends IndexShardTestCase {
null,
null,
new InternalEngineFactory(),
new EngineConfigFactory(shard.indexSettings()),
() -> {},
RetentionLeaseSyncer.EMPTY,
EMPTY_EVENT_LISTENER

View File

@ -82,6 +82,7 @@ import org.opensearch.common.xcontent.XContentType;
import org.opensearch.index.Index;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.engine.DocIdSeqNoAndSource;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.engine.InternalEngineFactory;
import org.opensearch.index.seqno.GlobalCheckpointSyncAction;
@ -228,6 +229,10 @@ public abstract class OpenSearchIndexLevelReplicationTestCase extends IndexShard
return new InternalEngineFactory();
}
protected EngineConfigFactory getEngineConfigFactory(IndexSettings indexSettings) {
return new EngineConfigFactory(indexSettings);
}
public int indexDocs(final int numOfDoc) throws Exception {
for (int doc = 0; doc < numOfDoc; doc++) {
final IndexRequest indexRequest = new IndexRequest(index.getName(), "type", Integer.toString(docId.incrementAndGet()))
@ -345,6 +350,7 @@ public abstract class OpenSearchIndexLevelReplicationTestCase extends IndexShard
final IndexShard newReplica =
newShard(shardRouting, shardPath, indexMetadata, null, null, getEngineFactory(shardRouting),
getEngineConfigFactory(new IndexSettings(indexMetadata, indexMetadata.getSettings())),
() -> {}, retentionLeaseSyncer, EMPTY_EVENT_LISTENER);
replicas.add(newReplica);
if (replicationTargets != null) {

View File

@ -67,6 +67,7 @@ import org.opensearch.index.cache.IndexCache;
import org.opensearch.index.cache.query.DisabledQueryCache;
import org.opensearch.index.engine.DocIdSeqNoAndSource;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.EngineConfigFactory;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.index.engine.EngineTestCase;
import org.opensearch.index.engine.InternalEngineFactory;
@ -353,7 +354,8 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
final ShardId shardId = routing.shardId();
final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir());
ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId);
return newShard(routing, shardPath, indexMetadata, null, indexReaderWrapper, engineFactory, globalCheckpointSyncer,
return newShard(routing, shardPath, indexMetadata, null, indexReaderWrapper, engineFactory,
new EngineConfigFactory(new IndexSettings(indexMetadata, indexMetadata.getSettings())), globalCheckpointSyncer,
retentionLeaseSyncer, EMPTY_EVENT_LISTENER, listeners);
}
@ -371,7 +373,7 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetadata indexMetadata,
@Nullable CheckedFunction<IndexSettings, Store, IOException> storeProvider,
@Nullable CheckedFunction<DirectoryReader, DirectoryReader, IOException> indexReaderWrapper,
@Nullable EngineFactory engineFactory,
@Nullable EngineFactory engineFactory, @Nullable EngineConfigFactory engineConfigFactory,
Runnable globalCheckpointSyncer, RetentionLeaseSyncer retentionLeaseSyncer,
IndexEventListener indexEventListener, IndexingOperationListener... listeners) throws IOException {
final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build();
@ -403,6 +405,7 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
mapperService,
similarityService,
engineFactory,
engineConfigFactory,
indexEventListener,
indexReaderWrapper,
threadPool,
@ -442,7 +445,8 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
* @param listeners new listerns to use for the newly created shard
*/
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
return reinitShard(current, routing, current.indexSettings.getIndexMetadata(), current.engineFactory, listeners);
return reinitShard(current, routing, current.indexSettings.getIndexMetadata(), current.engineFactory,
current.engineConfigFactory, listeners);
}
/**
@ -454,7 +458,7 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
* @param engineFactory the engine factory for the new shard
*/
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexMetadata indexMetadata, EngineFactory engineFactory,
IndexingOperationListener... listeners) throws IOException {
EngineConfigFactory engineConfigFactory, IndexingOperationListener... listeners) throws IOException {
closeShards(current);
return newShard(
routing,
@ -463,6 +467,7 @@ public abstract class IndexShardTestCase extends OpenSearchTestCase {
null,
null,
engineFactory,
engineConfigFactory,
current.getGlobalCheckpointSyncer(),
current.getRetentionLeaseSyncer(),
EMPTY_EVENT_LISTENER, listeners);