Переглянути джерело

Revert semantic_text model registry changes (#127075)

Kathleen DeRusso 6 місяців тому
батько
коміт
e280aa5d50

+ 0 - 5
docs/changelog/126629.yaml

@@ -1,5 +0,0 @@
-pr: 126629
-summary: Default new `semantic_text` fields to use BBQ when models are compatible
-area: Relevance
-type: enhancement
-issues: []

+ 0 - 1
server/src/main/java/org/elasticsearch/index/IndexVersions.java

@@ -160,7 +160,6 @@ public class IndexVersions {
     public static final IndexVersion SYNTHETIC_SOURCE_STORE_ARRAYS_NATIVELY_SCALED_FLOAT = def(9_020_0_00, Version.LUCENE_10_1_0);
     public static final IndexVersion USE_LUCENE101_POSTINGS_FORMAT = def(9_021_0_00, Version.LUCENE_10_1_0);
     public static final IndexVersion UPGRADE_TO_LUCENE_10_2_0 = def(9_022_00_0, Version.LUCENE_10_2_0);
-    public static final IndexVersion SEMANTIC_TEXT_DEFAULTS_TO_BBQ = def(9_023_0_00, Version.LUCENE_10_2_0);
     /*
      * STOP! READ THIS FIRST! No, really,
      *        ____ _____ ___  ____  _        ____  _____    _    ____    _____ _   _ ___ ____    _____ ___ ____  ____ _____ _

+ 20 - 51
server/src/main/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapper.java

@@ -290,11 +290,6 @@ public class DenseVectorFieldMapper extends FieldMapper {
             return this;
         }
 
-        public Builder indexOptions(IndexOptions indexOptions) {
-            this.indexOptions.setValue(indexOptions);
-            return this;
-        }
-
         @Override
         public DenseVectorFieldMapper build(MapperBuilderContext context) {
             // Validate again here because the dimensions or element type could have been set programmatically,
@@ -1226,7 +1221,7 @@ public class DenseVectorFieldMapper extends FieldMapper {
         public abstract VectorSimilarityFunction vectorSimilarityFunction(IndexVersion indexVersion, ElementType elementType);
     }
 
-    public abstract static class IndexOptions implements ToXContent {
+    abstract static class IndexOptions implements ToXContent {
         final VectorIndexType type;
 
         IndexOptions(VectorIndexType type) {
@@ -1235,36 +1230,21 @@ public class DenseVectorFieldMapper extends FieldMapper {
 
         abstract KnnVectorsFormat getVectorsFormat(ElementType elementType);
 
-        public boolean validate(ElementType elementType, int dim, boolean throwOnError) {
-            return validateElementType(elementType, throwOnError) && validateDimension(dim, throwOnError);
-        }
-
-        public boolean validateElementType(ElementType elementType) {
-            return validateElementType(elementType, true);
-        }
-
-        final boolean validateElementType(ElementType elementType, boolean throwOnError) {
-            boolean validElementType = type.supportsElementType(elementType);
-            if (throwOnError && validElementType == false) {
+        final void validateElementType(ElementType elementType) {
+            if (type.supportsElementType(elementType) == false) {
                 throw new IllegalArgumentException(
                     "[element_type] cannot be [" + elementType.toString() + "] when using index type [" + type + "]"
                 );
             }
-            return validElementType;
         }
 
         abstract boolean updatableTo(IndexOptions update);
 
-        public boolean validateDimension(int dim) {
-            return validateDimension(dim, true);
-        }
-
-        public boolean validateDimension(int dim, boolean throwOnError) {
-            boolean supportsDimension = type.supportsDimension(dim);
-            if (throwOnError && supportsDimension == false) {
-                throw new IllegalArgumentException(type.name + " only supports even dimensions; provided=" + dim);
+        public void validateDimension(int dim) {
+            if (type.supportsDimension(dim)) {
+                return;
             }
-            return supportsDimension;
+            throw new IllegalArgumentException(type.name + " only supports even dimensions; provided=" + dim);
         }
 
         abstract boolean doEquals(IndexOptions other);
@@ -1767,12 +1747,12 @@ public class DenseVectorFieldMapper extends FieldMapper {
 
     }
 
-    public static class Int8HnswIndexOptions extends QuantizedIndexOptions {
+    static class Int8HnswIndexOptions extends QuantizedIndexOptions {
         private final int m;
         private final int efConstruction;
         private final Float confidenceInterval;
 
-        public Int8HnswIndexOptions(int m, int efConstruction, Float confidenceInterval, RescoreVector rescoreVector) {
+        Int8HnswIndexOptions(int m, int efConstruction, Float confidenceInterval, RescoreVector rescoreVector) {
             super(VectorIndexType.INT8_HNSW, rescoreVector);
             this.m = m;
             this.efConstruction = efConstruction;
@@ -1910,11 +1890,11 @@ public class DenseVectorFieldMapper extends FieldMapper {
         }
     }
 
-    public static class BBQHnswIndexOptions extends QuantizedIndexOptions {
+    static class BBQHnswIndexOptions extends QuantizedIndexOptions {
         private final int m;
         private final int efConstruction;
 
-        public BBQHnswIndexOptions(int m, int efConstruction, RescoreVector rescoreVector) {
+        BBQHnswIndexOptions(int m, int efConstruction, RescoreVector rescoreVector) {
             super(VectorIndexType.BBQ_HNSW, rescoreVector);
             this.m = m;
             this.efConstruction = efConstruction;
@@ -1956,14 +1936,11 @@ public class DenseVectorFieldMapper extends FieldMapper {
         }
 
         @Override
-        public boolean validateDimension(int dim, boolean throwOnError) {
-            boolean supportsDimension = type.supportsDimension(dim);
-            if (throwOnError && supportsDimension == false) {
-                throw new IllegalArgumentException(
-                    type.name + " does not support dimensions fewer than " + BBQ_MIN_DIMS + "; provided=" + dim
-                );
+        public void validateDimension(int dim) {
+            if (type.supportsDimension(dim)) {
+                return;
             }
-            return supportsDimension;
+            throw new IllegalArgumentException(type.name + " does not support dimensions fewer than " + BBQ_MIN_DIMS + "; provided=" + dim);
         }
     }
 
@@ -2007,19 +1984,15 @@ public class DenseVectorFieldMapper extends FieldMapper {
         }
 
         @Override
-        public boolean validateDimension(int dim, boolean throwOnError) {
-            boolean supportsDimension = type.supportsDimension(dim);
-            if (throwOnError && supportsDimension == false) {
-                throw new IllegalArgumentException(
-                    type.name + " does not support dimensions fewer than " + BBQ_MIN_DIMS + "; provided=" + dim
-                );
+        public void validateDimension(int dim) {
+            if (type.supportsDimension(dim)) {
+                return;
             }
-            return supportsDimension;
+            throw new IllegalArgumentException(type.name + " does not support dimensions fewer than " + BBQ_MIN_DIMS + "; provided=" + dim);
         }
-
     }
 
-    public record RescoreVector(float oversample) implements ToXContentObject {
+    record RescoreVector(float oversample) implements ToXContentObject {
         static final String NAME = "rescore_vector";
         static final String OVERSAMPLE = "oversample";
 
@@ -2338,10 +2311,6 @@ public class DenseVectorFieldMapper extends FieldMapper {
         ElementType getElementType() {
             return elementType;
         }
-
-        public IndexOptions getIndexOptions() {
-            return indexOptions;
-        }
     }
 
     private final IndexOptions indexOptions;

+ 6 - 1
server/src/main/java/org/elasticsearch/inference/MinimalServiceSettings.java

@@ -249,6 +249,10 @@ public record MinimalServiceSettings(
         }
     }
 
+    public ModelConfigurations toModelConfigurations(String inferenceEntityId) {
+        return new ModelConfigurations(inferenceEntityId, taskType, service == null ? UNKNOWN_SERVICE : service, this);
+    }
+
     /**
      * Checks if the given {@link MinimalServiceSettings} is equivalent to the current definition.
      */
@@ -256,6 +260,7 @@ public record MinimalServiceSettings(
         return taskType == other.taskType
             && Objects.equals(dimensions, other.dimensions)
             && similarity == other.similarity
-            && elementType == other.elementType;
+            && elementType == other.elementType
+            && (service == null || service.equals(other.service));
     }
 }

+ 0 - 7
test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java

@@ -207,13 +207,6 @@ public abstract class MapperServiceTestCase extends FieldTypeTestCase {
         return mapperService;
     }
 
-    protected final MapperService createMapperService(IndexVersion indexVersion, Settings settings, XContentBuilder mappings)
-        throws IOException {
-        MapperService mapperService = createMapperService(indexVersion, settings, () -> true, mappings);
-        merge(mapperService, mappings);
-        return mapperService;
-    }
-
     protected final MapperService createMapperService(IndexVersion version, XContentBuilder mapping) throws IOException {
         return createMapperService(version, getIndexSettings(), () -> true, mapping);
     }

+ 7 - 13
x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java

@@ -197,7 +197,6 @@ public class InferencePlugin extends Plugin
     private final SetOnce<ElasticInferenceServiceComponents> elasticInferenceServiceComponents = new SetOnce<>();
     private final SetOnce<InferenceServiceRegistry> inferenceServiceRegistry = new SetOnce<>();
     private final SetOnce<ShardBulkInferenceActionFilter> shardBulkInferenceActionFilter = new SetOnce<>();
-    private final SetOnce<ModelRegistry> modelRegistry = new SetOnce<>();
     private List<InferenceServiceExtension> inferenceServiceExtensions;
 
     public InferencePlugin(Settings settings) {
@@ -261,8 +260,8 @@ public class InferencePlugin extends Plugin
         var amazonBedrockRequestSenderFactory = new AmazonBedrockRequestSender.Factory(serviceComponents.get(), services.clusterService());
         amazonBedrockFactory.set(amazonBedrockRequestSenderFactory);
 
-        modelRegistry.set(new ModelRegistry(services.clusterService(), services.client()));
-        services.clusterService().addListener(modelRegistry.get());
+        ModelRegistry modelRegistry = new ModelRegistry(services.clusterService(), services.client());
+        services.clusterService().addListener(modelRegistry);
 
         if (inferenceServiceExtensions == null) {
             inferenceServiceExtensions = new ArrayList<>();
@@ -300,7 +299,7 @@ public class InferencePlugin extends Plugin
                     elasicInferenceServiceFactory.get(),
                     serviceComponents.get(),
                     inferenceServiceSettings,
-                    modelRegistry.get(),
+                    modelRegistry,
                     authorizationHandler
                 )
             )
@@ -318,14 +317,14 @@ public class InferencePlugin extends Plugin
         var serviceRegistry = new InferenceServiceRegistry(inferenceServices, factoryContext);
         serviceRegistry.init(services.client());
         for (var service : serviceRegistry.getServices().values()) {
-            service.defaultConfigIds().forEach(modelRegistry.get()::addDefaultIds);
+            service.defaultConfigIds().forEach(modelRegistry::addDefaultIds);
         }
         inferenceServiceRegistry.set(serviceRegistry);
 
         var actionFilter = new ShardBulkInferenceActionFilter(
             services.clusterService(),
             serviceRegistry,
-            modelRegistry.get(),
+            modelRegistry,
             getLicenseState(),
             services.indexingPressure()
         );
@@ -335,7 +334,7 @@ public class InferencePlugin extends Plugin
         var inferenceStats = new PluginComponentBinding<>(InferenceStats.class, InferenceStats.create(meterRegistry));
 
         components.add(serviceRegistry);
-        components.add(modelRegistry.get());
+        components.add(modelRegistry);
         components.add(httpClientManager);
         components.add(inferenceStats);
 
@@ -499,16 +498,11 @@ public class InferencePlugin extends Plugin
         return Map.of(SemanticInferenceMetadataFieldsMapper.NAME, SemanticInferenceMetadataFieldsMapper.PARSER);
     }
 
-    // Overridable for tests
-    protected Supplier<ModelRegistry> getModelRegistry() {
-        return () -> modelRegistry.get();
-    }
-
     @Override
     public Map<String, Mapper.TypeParser> getMappers() {
         return Map.of(
             SemanticTextFieldMapper.CONTENT_TYPE,
-            SemanticTextFieldMapper.parser(getModelRegistry()),
+            SemanticTextFieldMapper.PARSER,
             OffsetSourceFieldMapper.CONTENT_TYPE,
             OffsetSourceFieldMapper.PARSER
         );

+ 8 - 78
x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java

@@ -7,9 +7,6 @@
 
 package org.elasticsearch.xpack.inference.mapper;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -21,7 +18,6 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.util.BitSet;
-import org.elasticsearch.ResourceNotFoundException;
 import org.elasticsearch.cluster.metadata.InferenceFieldMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
@@ -79,7 +75,6 @@ import org.elasticsearch.xpack.core.ml.inference.results.MlTextEmbeddingResults;
 import org.elasticsearch.xpack.core.ml.inference.results.TextExpansionResults;
 import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryBuilder;
 import org.elasticsearch.xpack.inference.highlight.SemanticTextHighlighter;
-import org.elasticsearch.xpack.inference.registry.ModelRegistry;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
@@ -94,9 +89,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
-import java.util.function.Supplier;
 
-import static org.elasticsearch.index.IndexVersions.SEMANTIC_TEXT_DEFAULTS_TO_BBQ;
 import static org.elasticsearch.inference.TaskType.SPARSE_EMBEDDING;
 import static org.elasticsearch.inference.TaskType.TEXT_EMBEDDING;
 import static org.elasticsearch.search.SearchService.DEFAULT_SIZE;
@@ -119,7 +112,6 @@ import static org.elasticsearch.xpack.inference.services.elasticsearch.Elasticse
  * A {@link FieldMapper} for semantic text fields.
  */
 public class SemanticTextFieldMapper extends FieldMapper implements InferenceFieldMapper {
-    private static final Logger logger = LogManager.getLogger(SemanticTextFieldMapper.class);
     public static final NodeFeature SEMANTIC_TEXT_IN_OBJECT_FIELD_FIX = new NodeFeature("semantic_text.in_object_field_fix");
     public static final NodeFeature SEMANTIC_TEXT_SINGLE_FIELD_UPDATE_FIX = new NodeFeature("semantic_text.single_field_update_fix");
     public static final NodeFeature SEMANTIC_TEXT_DELETE_FIX = new NodeFeature("semantic_text.delete_fix");
@@ -135,14 +127,10 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
     public static final String CONTENT_TYPE = "semantic_text";
     public static final String DEFAULT_ELSER_2_INFERENCE_ID = DEFAULT_ELSER_ID;
 
-    public static final float DEFAULT_RESCORE_OVERSAMPLE = 3.0f;
-
-    public static final TypeParser parser(Supplier<ModelRegistry> modelRegistry) {
-        return new TypeParser(
-            (n, c) -> new Builder(n, c::bitSetProducer, c.getIndexSettings(), modelRegistry.get()),
-            List.of(validateParserContext(CONTENT_TYPE))
-        );
-    }
+    public static final TypeParser PARSER = new TypeParser(
+        (n, c) -> new Builder(n, c::bitSetProducer, c.getIndexSettings()),
+        List.of(validateParserContext(CONTENT_TYPE))
+    );
 
     public static BiConsumer<String, MappingParserContext> validateParserContext(String type) {
         return (n, c) -> {
@@ -154,7 +142,6 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
     }
 
     public static class Builder extends FieldMapper.Builder {
-        private final ModelRegistry modelRegistry;
         private final boolean useLegacyFormat;
 
         private final Parameter<String> inferenceId = Parameter.stringParam(
@@ -212,21 +199,14 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
             Builder builder = new Builder(
                 mapper.leafName(),
                 mapper.fieldType().getChunksField().bitsetProducer(),
-                mapper.fieldType().getChunksField().indexSettings(),
-                mapper.modelRegistry
+                mapper.fieldType().getChunksField().indexSettings()
             );
             builder.init(mapper);
             return builder;
         }
 
-        public Builder(
-            String name,
-            Function<Query, BitSetProducer> bitSetProducer,
-            IndexSettings indexSettings,
-            ModelRegistry modelRegistry
-        ) {
+        public Builder(String name, Function<Query, BitSetProducer> bitSetProducer, IndexSettings indexSettings) {
             super(name);
-            this.modelRegistry = modelRegistry;
             this.useLegacyFormat = InferenceMetadataFieldsMapper.isEnabled(indexSettings.getSettings()) == false;
             this.inferenceFieldBuilder = c -> createInferenceField(
                 c,
@@ -284,32 +264,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
             if (useLegacyFormat && multiFieldsBuilder.hasMultiFields()) {
                 throw new IllegalArgumentException(CONTENT_TYPE + " field [" + leafName() + "] does not support multi-fields");
             }
-
-            if (modelSettings.get() == null) {
-                try {
-                    var resolvedModelSettings = modelRegistry.getMinimalServiceSettings(inferenceId.get());
-                    if (resolvedModelSettings != null) {
-                        modelSettings.setValue(resolvedModelSettings);
-                    }
-                } catch (ResourceNotFoundException exc) {
-                    // We allow the inference ID to be unregistered at this point.
-                    // This will delay the creation of sub-fields, so indexing and querying for this field won't work
-                    // until the corresponding inference endpoint is created.
-                }
-            }
-
             if (modelSettings.get() != null) {
                 validateServiceSettings(modelSettings.get());
-            } else {
-                logger.warn(
-                    "The field [{}] references an unknown inference ID [{}]. "
-                        + "Indexing and querying this field will not work correctly until the corresponding "
-                        + "inference endpoint is created.",
-                    leafName(),
-                    inferenceId.get()
-                );
             }
-
             final String fullName = context.buildFullName(leafName());
 
             if (context.isInNestedContext()) {
@@ -330,8 +287,7 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
                     useLegacyFormat,
                     meta.getValue()
                 ),
-                builderParams(this, context),
-                modelRegistry
+                builderParams(this, context)
             );
         }
 
@@ -372,17 +328,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
         }
     }
 
-    private final ModelRegistry modelRegistry;
-
-    private SemanticTextFieldMapper(
-        String simpleName,
-        MappedFieldType mappedFieldType,
-        BuilderParams builderParams,
-        ModelRegistry modelRegistry
-    ) {
+    private SemanticTextFieldMapper(String simpleName, MappedFieldType mappedFieldType, BuilderParams builderParams) {
         super(simpleName, mappedFieldType, builderParams);
         ensureMultiFields(builderParams.multiFields().iterator());
-        this.modelRegistry = modelRegistry;
     }
 
     private void ensureMultiFields(Iterator<FieldMapper> mappers) {
@@ -1058,30 +1006,12 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
                 denseVectorMapperBuilder.dimensions(modelSettings.dimensions());
                 denseVectorMapperBuilder.elementType(modelSettings.elementType());
 
-                DenseVectorFieldMapper.IndexOptions defaultIndexOptions = null;
-                if (indexVersionCreated.onOrAfter(SEMANTIC_TEXT_DEFAULTS_TO_BBQ)) {
-                    defaultIndexOptions = defaultSemanticDenseIndexOptions();
-                }
-                if (defaultIndexOptions != null
-                    && defaultIndexOptions.validate(modelSettings.elementType(), modelSettings.dimensions(), false)) {
-                    denseVectorMapperBuilder.indexOptions(defaultIndexOptions);
-                }
-
                 yield denseVectorMapperBuilder;
             }
             default -> throw new IllegalArgumentException("Invalid task_type in model_settings [" + modelSettings.taskType().name() + "]");
         };
     }
 
-    static DenseVectorFieldMapper.IndexOptions defaultSemanticDenseIndexOptions() {
-        // As embedding models for text perform better with BBQ, we aggressively default semantic_text fields to use optimized index
-        // options outside of dense_vector defaults
-        int m = Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN;
-        int efConstruction = Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
-        DenseVectorFieldMapper.RescoreVector rescoreVector = new DenseVectorFieldMapper.RescoreVector(DEFAULT_RESCORE_OVERSAMPLE);
-        return new DenseVectorFieldMapper.BBQHnswIndexOptions(m, efConstruction, rescoreVector);
-    }
-
     private static boolean canMergeModelSettings(MinimalServiceSettings previous, MinimalServiceSettings current, Conflicts conflicts) {
         if (previous != null && current != null && previous.canMergeWith(current)) {
             return true;

+ 8 - 26
x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/registry/ModelRegistry.java

@@ -36,7 +36,6 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateAckListener;
 import org.elasticsearch.cluster.ClusterStateListener;
 import org.elasticsearch.cluster.SimpleBatchedAckListenerTaskExecutor;
-import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -141,6 +140,7 @@ public class ModelRegistry implements ClusterStateListener {
     private static final String MODEL_ID_FIELD = "model_id";
     private static final Logger logger = LogManager.getLogger(ModelRegistry.class);
 
+    private final ClusterService clusterService;
     private final OriginSettingClient client;
     private final Map<String, InferenceService.DefaultConfigId> defaultConfigIds;
 
@@ -148,11 +148,10 @@ public class ModelRegistry implements ClusterStateListener {
     private final AtomicBoolean upgradeMetadataInProgress = new AtomicBoolean(false);
     private final Set<String> preventDeletionLock = Collections.newSetFromMap(new ConcurrentHashMap<>());
 
-    private volatile Metadata lastMetadata;
-
     public ModelRegistry(ClusterService clusterService, Client client) {
         this.client = new OriginSettingClient(client, ClientHelper.INFERENCE_ORIGIN);
         this.defaultConfigIds = new ConcurrentHashMap<>();
+        this.clusterService = clusterService;
         var executor = new SimpleBatchedAckListenerTaskExecutor<MetadataTask>() {
             @Override
             public Tuple<ClusterState, ClusterStateAckListener> executeTask(MetadataTask task, ClusterState clusterState) throws Exception {
@@ -225,18 +224,11 @@ public class ModelRegistry implements ClusterStateListener {
      * @throws ResourceNotFoundException if the specified id is guaranteed to not exist in the cluster.
      */
     public MinimalServiceSettings getMinimalServiceSettings(String inferenceEntityId) throws ResourceNotFoundException {
-        synchronized (this) {
-            assert lastMetadata != null : "initial cluster state not set yet";
-            if (lastMetadata == null) {
-                throw new IllegalStateException("initial cluster state not set yet");
-            }
-        }
         var config = defaultConfigIds.get(inferenceEntityId);
         if (config != null) {
             return config.settings();
         }
-        var project = lastMetadata.getProject(ProjectId.DEFAULT);
-        var state = ModelRegistryMetadata.fromState(project);
+        var state = ModelRegistryMetadata.fromState(clusterService.state().projectState().metadata());
         var existing = state.getMinimalServiceSettings(inferenceEntityId);
         if (state.isUpgraded() && existing == null) {
             throw new ResourceNotFoundException(inferenceEntityId + " does not exist in this cluster.");
@@ -692,14 +684,10 @@ public class ModelRegistry implements ClusterStateListener {
                 if (updateClusterState) {
                     var storeListener = getStoreMetadataListener(inferenceEntityId, listener);
                     try {
+                        var projectId = clusterService.state().projectState().projectId();
                         metadataTaskQueue.submitTask(
                             "add model [" + inferenceEntityId + "]",
-                            new AddModelMetadataTask(
-                                ProjectId.DEFAULT,
-                                inferenceEntityId,
-                                new MinimalServiceSettings(model),
-                                storeListener
-                            ),
+                            new AddModelMetadataTask(projectId, inferenceEntityId, new MinimalServiceSettings(model), storeListener),
                             timeout
                         );
                     } catch (Exception exc) {
@@ -866,9 +854,10 @@ public class ModelRegistry implements ClusterStateListener {
                     }
                 };
                 try {
+                    var projectId = clusterService.state().projectState().projectId();
                     metadataTaskQueue.submitTask(
                         "delete models [" + inferenceEntityIds + "]",
-                        new DeleteModelMetadataTask(ProjectId.DEFAULT, inferenceEntityIds, clusterStateListener),
+                        new DeleteModelMetadataTask(projectId, inferenceEntityIds, clusterStateListener),
                         null
                     );
                 } catch (Exception exc) {
@@ -946,13 +935,6 @@ public class ModelRegistry implements ClusterStateListener {
 
     @Override
     public void clusterChanged(ClusterChangedEvent event) {
-        if (lastMetadata == null || event.metadataChanged()) {
-            // keep track of the last applied cluster state
-            synchronized (this) {
-                lastMetadata = event.state().metadata();
-            }
-        }
-
         if (event.localNodeMaster() == false) {
             return;
         }
@@ -1002,7 +984,7 @@ public class ModelRegistry implements ClusterStateListener {
                     metadataTaskQueue.submitTask(
                         "model registry auto upgrade",
                         new UpgradeModelsMetadataTask(
-                            ProjectId.DEFAULT,
+                            clusterService.state().metadata().getProject().id(),
                             map,
                             ActionListener.running(() -> upgradeMetadataInProgress.set(false))
                         ),

+ 7 - 11
x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticInferenceMetadataFieldsMapperTests.java

@@ -37,10 +37,7 @@ public class SemanticInferenceMetadataFieldsMapperTests extends MapperServiceTes
         assertFalse(InferenceMetadataFieldsMapper.isEnabled(settings));
 
         settings = Settings.builder()
-            .put(
-                IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(),
-                getRandomCompatibleIndexVersion(true, IndexVersionUtils.getPreviousVersion(IndexVersions.INFERENCE_METADATA_FIELDS))
-            )
+            .put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), getRandomCompatibleIndexVersion(true))
             .put(InferenceMetadataFieldsMapper.USE_LEGACY_SEMANTIC_TEXT_FORMAT.getKey(), false)
             .build();
         assertFalse(InferenceMetadataFieldsMapper.isEnabled(settings));
@@ -117,18 +114,18 @@ public class SemanticInferenceMetadataFieldsMapperTests extends MapperServiceTes
     }
 
     static IndexVersion getRandomCompatibleIndexVersion(boolean useLegacyFormat) {
-        return getRandomCompatibleIndexVersion(useLegacyFormat, IndexVersion.current());
-    }
-
-    static IndexVersion getRandomCompatibleIndexVersion(boolean useLegacyFormat, IndexVersion maxVersion) {
         if (useLegacyFormat) {
             if (randomBoolean()) {
-                return IndexVersionUtils.randomVersionBetween(random(), IndexVersions.UPGRADE_TO_LUCENE_10_0_0, maxVersion);
+                return IndexVersionUtils.randomVersionBetween(
+                    random(),
+                    IndexVersions.UPGRADE_TO_LUCENE_10_0_0,
+                    IndexVersionUtils.getPreviousVersion(IndexVersions.INFERENCE_METADATA_FIELDS)
+                );
             }
             return IndexVersionUtils.randomPreviousCompatibleVersion(random(), IndexVersions.INFERENCE_METADATA_FIELDS_BACKPORT);
         } else {
             if (randomBoolean()) {
-                return IndexVersionUtils.randomVersionBetween(random(), IndexVersions.INFERENCE_METADATA_FIELDS, maxVersion);
+                return IndexVersionUtils.randomVersionBetween(random(), IndexVersions.INFERENCE_METADATA_FIELDS, IndexVersion.current());
             }
             return IndexVersionUtils.randomVersionBetween(
                 random(),
@@ -137,5 +134,4 @@ public class SemanticInferenceMetadataFieldsMapperTests extends MapperServiceTes
             );
         }
     }
-
 }

+ 11 - 155
x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapperTests.java

@@ -9,7 +9,6 @@ package org.elasticsearch.xpack.inference.mapper;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 
-import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexableField;
@@ -25,7 +24,6 @@ import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.search.join.QueryBitSetProducer;
 import org.apache.lucene.search.join.ScoreMode;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.common.CheckedBiConsumer;
 import org.elasticsearch.common.CheckedBiFunction;
@@ -36,7 +34,6 @@ import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.CheckedConsumer;
 import org.elasticsearch.index.IndexVersion;
-import org.elasticsearch.index.IndexVersions;
 import org.elasticsearch.index.mapper.DocumentMapper;
 import org.elasticsearch.index.mapper.DocumentParsingException;
 import org.elasticsearch.index.mapper.FieldMapper;
@@ -66,10 +63,6 @@ import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.search.LeafNestedDocuments;
 import org.elasticsearch.search.NestedDocuments;
 import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.test.ClusterServiceUtils;
-import org.elasticsearch.test.client.NoOpClient;
-import org.elasticsearch.test.index.IndexVersionUtils;
-import org.elasticsearch.threadpool.TestThreadPool;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xcontent.json.JsonXContent;
@@ -77,10 +70,7 @@ import org.elasticsearch.xpack.core.XPackClientPlugin;
 import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryWrapper;
 import org.elasticsearch.xpack.inference.InferencePlugin;
 import org.elasticsearch.xpack.inference.model.TestModel;
-import org.elasticsearch.xpack.inference.registry.ModelRegistry;
-import org.junit.After;
 import org.junit.AssumptionViolatedException;
-import org.junit.Before;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -90,7 +80,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.function.BiConsumer;
-import java.util.function.Supplier;
 
 import static org.elasticsearch.xpack.inference.mapper.SemanticTextField.CHUNKED_EMBEDDINGS_FIELD;
 import static org.elasticsearch.xpack.inference.mapper.SemanticTextField.CHUNKS_FIELD;
@@ -112,22 +101,10 @@ import static org.hamcrest.Matchers.instanceOf;
 public class SemanticTextFieldMapperTests extends MapperTestCase {
     private final boolean useLegacyFormat;
 
-    private TestThreadPool threadPool;
-
     public SemanticTextFieldMapperTests(boolean useLegacyFormat) {
         this.useLegacyFormat = useLegacyFormat;
     }
 
-    @Before
-    private void startThreadPool() {
-        threadPool = createThreadPool();
-    }
-
-    @After
-    private void stopThreadPool() {
-        threadPool.close();
-    }
-
     @ParametersFactory
     public static Iterable<Object[]> parameters() throws Exception {
         return List.of(new Object[] { true }, new Object[] { false });
@@ -135,61 +112,18 @@ public class SemanticTextFieldMapperTests extends MapperTestCase {
 
     @Override
     protected Collection<? extends Plugin> getPlugins() {
-        var clusterService = ClusterServiceUtils.createClusterService(threadPool);
-        var modelRegistry = new ModelRegistry(clusterService, new NoOpClient(threadPool));
-        modelRegistry.clusterChanged(new ClusterChangedEvent("init", clusterService.state(), clusterService.state()) {
-            @Override
-            public boolean localNodeMaster() {
-                return false;
-            }
-        });
-        return List.of(new InferencePlugin(Settings.EMPTY) {
-            @Override
-            protected Supplier<ModelRegistry> getModelRegistry() {
-                return () -> modelRegistry;
-            }
-        }, new XPackClientPlugin());
+        return List.of(new InferencePlugin(Settings.EMPTY), new XPackClientPlugin());
     }
 
     private MapperService createMapperService(XContentBuilder mappings, boolean useLegacyFormat) throws IOException {
-        IndexVersion indexVersion = SemanticInferenceMetadataFieldsMapperTests.getRandomCompatibleIndexVersion(useLegacyFormat);
-        return createMapperService(mappings, useLegacyFormat, indexVersion, indexVersion, false);
-    }
-
-    private MapperService createMapperService(XContentBuilder mappings, boolean useLegacyFormat, IndexVersion minIndexVersion)
-        throws IOException {
-        return createMapperService(mappings, useLegacyFormat, minIndexVersion, IndexVersion.current(), false);
-    }
-
-    private MapperService createMapperService(
-        XContentBuilder mappings,
-        boolean useLegacyFormat,
-        IndexVersion minIndexVersion,
-        IndexVersion maxIndexVersion,
-        boolean propagateIndexVersion
-    ) throws IOException {
-        validateIndexVersion(minIndexVersion, useLegacyFormat);
-        IndexVersion indexVersion = IndexVersionUtils.randomVersionBetween(random(), minIndexVersion, maxIndexVersion);
         var settings = Settings.builder()
-            .put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), indexVersion)
+            .put(
+                IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(),
+                SemanticInferenceMetadataFieldsMapperTests.getRandomCompatibleIndexVersion(useLegacyFormat)
+            )
             .put(InferenceMetadataFieldsMapper.USE_LEGACY_SEMANTIC_TEXT_FORMAT.getKey(), useLegacyFormat)
             .build();
-        // TODO - This is added, because we discovered a bug where the index version was not being correctly propagated
-        // in our mappings even though we were specifying the index version in settings. We will fix this in a followup and
-        // remove the boolean flag accordingly.
-        if (propagateIndexVersion) {
-            return createMapperService(indexVersion, settings, mappings);
-        } else {
-            return createMapperService(settings, mappings);
-        }
-    }
-
-    private static void validateIndexVersion(IndexVersion indexVersion, boolean useLegacyFormat) {
-        if (useLegacyFormat == false
-            && indexVersion.before(IndexVersions.INFERENCE_METADATA_FIELDS)
-            && indexVersion.between(IndexVersions.INFERENCE_METADATA_FIELDS_BACKPORT, IndexVersions.UPGRADE_TO_LUCENE_10_0_0) == false) {
-            throw new IllegalArgumentException("Index version " + indexVersion + " does not support new semantic text format");
-        }
+        return createMapperService(settings, mappings);
     }
 
     @Override
@@ -635,15 +569,14 @@ public class SemanticTextFieldMapperTests extends MapperTestCase {
     }
 
     private static void assertSemanticTextField(MapperService mapperService, String fieldName, boolean expectedModelSettings) {
-        assertSemanticTextField(mapperService, fieldName, expectedModelSettings, null, null);
+        assertSemanticTextField(mapperService, fieldName, expectedModelSettings, null);
     }
 
     private static void assertSemanticTextField(
         MapperService mapperService,
         String fieldName,
         boolean expectedModelSettings,
-        ChunkingSettings expectedChunkingSettings,
-        DenseVectorFieldMapper.IndexOptions expectedIndexOptions
+        ChunkingSettings expectedChunkingSettings
     ) {
         Mapper mapper = mapperService.mappingLookup().getMapper(fieldName);
         assertNotNull(mapper);
@@ -689,17 +622,8 @@ public class SemanticTextFieldMapperTests extends MapperTestCase {
                     assertThat(embeddingsMapper, instanceOf(SparseVectorFieldMapper.class));
                     SparseVectorFieldMapper sparseMapper = (SparseVectorFieldMapper) embeddingsMapper;
                     assertEquals(sparseMapper.fieldType().isStored(), semanticTextFieldType.useLegacyFormat() == false);
-                    assertNull(expectedIndexOptions);
-                }
-                case TEXT_EMBEDDING -> {
-                    assertThat(embeddingsMapper, instanceOf(DenseVectorFieldMapper.class));
-                    DenseVectorFieldMapper denseVectorFieldMapper = (DenseVectorFieldMapper) embeddingsMapper;
-                    if (expectedIndexOptions != null) {
-                        assertEquals(expectedIndexOptions, denseVectorFieldMapper.fieldType().getIndexOptions());
-                    } else {
-                        assertNull(denseVectorFieldMapper.fieldType().getIndexOptions());
-                    }
                 }
+                case TEXT_EMBEDDING -> assertThat(embeddingsMapper, instanceOf(DenseVectorFieldMapper.class));
                 default -> throw new AssertionError("Invalid task type");
             }
         } else {
@@ -994,11 +918,11 @@ public class SemanticTextFieldMapperTests extends MapperTestCase {
             mapping(b -> addSemanticTextMapping(b, fieldName, model.getInferenceEntityId(), null, chunkingSettings)),
             useLegacyFormat
         );
-        assertSemanticTextField(mapperService, fieldName, false, chunkingSettings, null);
+        assertSemanticTextField(mapperService, fieldName, false, chunkingSettings);
 
         ChunkingSettings newChunkingSettings = generateRandomChunkingSettingsOtherThan(chunkingSettings);
         merge(mapperService, mapping(b -> addSemanticTextMapping(b, fieldName, model.getInferenceEntityId(), null, newChunkingSettings)));
-        assertSemanticTextField(mapperService, fieldName, false, newChunkingSettings, null);
+        assertSemanticTextField(mapperService, fieldName, false, newChunkingSettings);
     }
 
     public void testModelSettingsRequiredWithChunks() throws IOException {
@@ -1128,74 +1052,6 @@ public class SemanticTextFieldMapperTests extends MapperTestCase {
         assertThat(existsQuery, instanceOf(ESToParentBlockJoinQuery.class));
     }
 
-    private static DenseVectorFieldMapper.IndexOptions defaultDenseVectorIndexOptions() {
-        // These are the default index options for dense_vector fields, and used for semantic_text fields incompatible with BBQ.
-        int m = Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN;
-        int efConstruction = Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH;
-        return new DenseVectorFieldMapper.Int8HnswIndexOptions(m, efConstruction, null, null);
-    }
-
-    public void testDefaultIndexOptions() throws IOException {
-
-        // We default to BBQ for eligible dense vectors
-        var mapperService = createMapperService(fieldMapping(b -> {
-            b.field("type", "semantic_text");
-            b.field("inference_id", "another_inference_id");
-            b.startObject("model_settings");
-            b.field("task_type", "text_embedding");
-            b.field("dimensions", 100);
-            b.field("similarity", "cosine");
-            b.field("element_type", "float");
-            b.endObject();
-        }), useLegacyFormat, IndexVersions.SEMANTIC_TEXT_DEFAULTS_TO_BBQ);
-        assertSemanticTextField(mapperService, "field", true, null, SemanticTextFieldMapper.defaultSemanticDenseIndexOptions());
-
-        // Element types that are incompatible with BBQ will continue to use dense_vector defaults
-        mapperService = createMapperService(fieldMapping(b -> {
-            b.field("type", "semantic_text");
-            b.field("inference_id", "another_inference_id");
-            b.startObject("model_settings");
-            b.field("task_type", "text_embedding");
-            b.field("dimensions", 100);
-            b.field("similarity", "cosine");
-            b.field("element_type", "byte");
-            b.endObject();
-        }), useLegacyFormat, IndexVersions.SEMANTIC_TEXT_DEFAULTS_TO_BBQ);
-        assertSemanticTextField(mapperService, "field", true, null, null);
-
-        // A dim count of 10 is too small to support BBQ, so we continue to use dense_vector defaults
-        mapperService = createMapperService(fieldMapping(b -> {
-            b.field("type", "semantic_text");
-            b.field("inference_id", "another_inference_id");
-            b.startObject("model_settings");
-            b.field("task_type", "text_embedding");
-            b.field("dimensions", 10);
-            b.field("similarity", "cosine");
-            b.field("element_type", "float");
-            b.endObject();
-        }), useLegacyFormat, IndexVersions.SEMANTIC_TEXT_DEFAULTS_TO_BBQ);
-        assertSemanticTextField(mapperService, "field", true, null, defaultDenseVectorIndexOptions());
-
-        // Previous index versions do not set BBQ index options
-        mapperService = createMapperService(fieldMapping(b -> {
-            b.field("type", "semantic_text");
-            b.field("inference_id", "another_inference_id");
-            b.startObject("model_settings");
-            b.field("task_type", "text_embedding");
-            b.field("dimensions", 100);
-            b.field("similarity", "cosine");
-            b.field("element_type", "float");
-            b.endObject();
-        }),
-            useLegacyFormat,
-            IndexVersions.INFERENCE_METADATA_FIELDS,
-            IndexVersionUtils.getPreviousVersion(IndexVersions.SEMANTIC_TEXT_DEFAULTS_TO_BBQ),
-            true
-        );
-        assertSemanticTextField(mapperService, "field", true, null, defaultDenseVectorIndexOptions());
-
-    }
-
     @Override
     protected void assertExistsQuery(MappedFieldType fieldType, Query query, LuceneDocument fields) {
         // Until a doc is indexed, the query is rewritten as match no docs

+ 1 - 41
x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilderTests.java

@@ -22,14 +22,12 @@ import org.elasticsearch.action.ActionRequest;
 import org.elasticsearch.action.ActionType;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
 import org.elasticsearch.client.internal.Client;
-import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.core.IOUtils;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.mapper.InferenceMetadataFieldsMapper;
 import org.elasticsearch.index.mapper.MapperService;
@@ -48,9 +46,6 @@ import org.elasticsearch.inference.SimilarityMeasure;
 import org.elasticsearch.inference.TaskType;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.AbstractQueryTestCase;
-import org.elasticsearch.test.ClusterServiceUtils;
-import org.elasticsearch.test.client.NoOpClient;
-import org.elasticsearch.threadpool.TestThreadPool;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xcontent.json.JsonXContent;
@@ -65,8 +60,6 @@ import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryWrapper;
 import org.elasticsearch.xpack.core.ml.search.WeightedToken;
 import org.elasticsearch.xpack.inference.InferencePlugin;
 import org.elasticsearch.xpack.inference.mapper.SemanticTextField;
-import org.elasticsearch.xpack.inference.registry.ModelRegistry;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
@@ -77,7 +70,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.function.Supplier;
 
 import static org.apache.lucene.search.BooleanClause.Occur.FILTER;
 import static org.apache.lucene.search.BooleanClause.Occur.MUST;
@@ -126,24 +118,6 @@ public class SemanticQueryBuilderTests extends AbstractQueryTestCase<SemanticQue
         useSearchInferenceId = randomBoolean();
     }
 
-    @BeforeClass
-    public static void startModelRegistry() {
-        threadPool = new TestThreadPool(SemanticQueryBuilderTests.class.getName());
-        var clusterService = ClusterServiceUtils.createClusterService(threadPool);
-        modelRegistry = new ModelRegistry(clusterService, new NoOpClient(threadPool));
-        modelRegistry.clusterChanged(new ClusterChangedEvent("init", clusterService.state(), clusterService.state()) {
-            @Override
-            public boolean localNodeMaster() {
-                return false;
-            }
-        });
-    }
-
-    @AfterClass
-    public static void stopModelRegistry() {
-        IOUtils.closeWhileHandlingException(threadPool);
-    }
-
     @Override
     @Before
     public void setUp() throws Exception {
@@ -153,7 +127,7 @@ public class SemanticQueryBuilderTests extends AbstractQueryTestCase<SemanticQue
 
     @Override
     protected Collection<Class<? extends Plugin>> getPlugins() {
-        return List.of(XPackClientPlugin.class, InferencePluginWithModelRegistry.class, FakeMlPlugin.class);
+        return List.of(XPackClientPlugin.class, InferencePlugin.class, FakeMlPlugin.class);
     }
 
     @Override
@@ -420,18 +394,4 @@ public class SemanticQueryBuilderTests extends AbstractQueryTestCase<SemanticQue
             return new MlInferenceNamedXContentProvider().getNamedWriteables();
         }
     }
-
-    private static TestThreadPool threadPool;
-    private static ModelRegistry modelRegistry;
-
-    public static class InferencePluginWithModelRegistry extends InferencePlugin {
-        public InferencePluginWithModelRegistry(Settings settings) {
-            super(settings);
-        }
-
-        @Override
-        protected Supplier<ModelRegistry> getModelRegistry() {
-            return () -> modelRegistry;
-        }
-    }
 }

+ 0 - 27
x-pack/plugin/inference/src/yamlRestTest/java/org/elasticsearch/xpack/inference/InferenceRestIT.java

@@ -9,20 +9,13 @@ package org.elasticsearch.xpack.inference;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 
-import org.elasticsearch.client.Request;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.test.cluster.ElasticsearchCluster;
 import org.elasticsearch.test.cluster.local.distribution.DistributionType;
 import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
 import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase;
-import org.junit.After;
 import org.junit.ClassRule;
 
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
 public class InferenceRestIT extends ESClientYamlSuiteTestCase {
 
     @ClassRule
@@ -57,24 +50,4 @@ public class InferenceRestIT extends ESClientYamlSuiteTestCase {
     public static Iterable<Object[]> parameters() throws Exception {
         return ESClientYamlSuiteTestCase.createParameters();
     }
-
-    @After
-    public void cleanup() throws Exception {
-        for (var model : getAllModels()) {
-            var inferenceId = model.get("inference_id");
-            try {
-                var endpoint = Strings.format("_inference/%s?force", inferenceId);
-                adminClient().performRequest(new Request("DELETE", endpoint));
-            } catch (Exception ex) {
-                logger.warn(() -> "failed to delete inference endpoint " + inferenceId, ex);
-            }
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    static List<Map<String, Object>> getAllModels() throws IOException {
-        var request = new Request("GET", "_inference/_all");
-        var response = client().performRequest(request);
-        return (List<Map<String, Object>>) entityAsMap(response).get("endpoints");
-    }
 }