浏览代码

[Transform] switch to a composable template for the audit index (#79032)

switch to a composable template for the transform audit index
Hendrik Muhs 4 年之前
父节点
当前提交
43a9d4d82b

+ 3 - 14
x-pack/plugin/src/test/java/org/elasticsearch/xpack/test/rest/AbstractXPackRestTest.java

@@ -7,15 +7,15 @@
 package org.elasticsearch.xpack.test.rest;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
+
 import org.apache.http.HttpStatus;
 import org.apache.lucene.util.TimeUnits;
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.core.CheckedFunction;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.common.xcontent.support.XContentMapValues;
+import org.elasticsearch.core.CheckedFunction;
 import org.elasticsearch.plugins.MetadataUpgrader;
 import org.elasticsearch.test.SecuritySettingsSourceField;
 import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
@@ -26,7 +26,6 @@ import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex;
 import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields;
 import org.elasticsearch.xpack.core.ml.notifications.NotificationsIndex;
 import org.elasticsearch.xpack.core.rollup.job.RollupJob;
-import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants;
 import org.junit.After;
 import org.junit.Before;
 
@@ -79,7 +78,7 @@ public class AbstractXPackRestTest extends ESClientYamlSuiteTestCase {
     }
 
     /**
-     * Waits for Machine Learning and Transform templates to be created by the {@link MetadataUpgrader}
+     * Waits for Machine Learning templates to be created by the {@link MetadataUpgrader}
      */
     private void waitForTemplates() {
         if (installTemplates()) {
@@ -94,16 +93,6 @@ public class AbstractXPackRestTest extends ESClientYamlSuiteTestCase {
                     response -> true,
                     () -> "Exception when waiting for [" + template + "] template to be created");
             }
-
-            List<String> legacyTemplates = Collections.singletonList(
-                TransformInternalIndexConstants.AUDIT_INDEX
-            );
-
-            for (String legacyTemplate : legacyTemplates) {
-                awaitCallApi("indices.exists_template", singletonMap("name", legacyTemplate), emptyList(),
-                    response -> true,
-                    () -> "Exception when waiting for [" + legacyTemplate + "] legacy template to be created");
-            }
         }
     }
 

+ 0 - 16
x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java

@@ -21,7 +21,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.client.OriginSettingClient;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
-import org.elasticsearch.cluster.metadata.IndexTemplateMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -72,7 +71,6 @@ import org.elasticsearch.xpack.core.transform.action.StopTransformAction;
 import org.elasticsearch.xpack.core.transform.action.UpdateTransformAction;
 import org.elasticsearch.xpack.core.transform.action.UpgradeTransformsAction;
 import org.elasticsearch.xpack.core.transform.action.ValidateTransformAction;
-import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants;
 import org.elasticsearch.xpack.transform.action.TransportDeleteTransformAction;
 import org.elasticsearch.xpack.transform.action.TransportGetTransformAction;
 import org.elasticsearch.xpack.transform.action.TransportGetTransformStatsAction;
@@ -108,9 +106,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.function.Supplier;
-import java.util.function.UnaryOperator;
 
 import static org.elasticsearch.xpack.core.ClientHelper.TRANSFORM_ORIGIN;
 import static org.elasticsearch.xpack.core.transform.TransformMessages.FAILED_TO_UNSET_RESET_MODE;
@@ -241,18 +237,6 @@ public class Transform extends Plugin implements SystemIndexPlugin, PersistentTa
         return Arrays.asList(transformServices.get(), new TransformClusterStateListener(clusterService, client));
     }
 
-    @Override
-    public UnaryOperator<Map<String, IndexTemplateMetadata>> getIndexTemplateMetadataUpgrader() {
-        return templates -> {
-            try {
-                templates.put(TransformInternalIndexConstants.AUDIT_INDEX, TransformInternalIndex.getAuditIndexTemplateMetadata());
-            } catch (IOException e) {
-                logger.warn("Error creating transform audit index", e);
-            }
-            return templates;
-        };
-    }
-
     @Override
     public List<PersistentTasksExecutor<?>> getPersistentTasksExecutor(
         ClusterService clusterService,

+ 35 - 8
x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/notifications/TransformAuditor.java

@@ -7,11 +7,16 @@
 package org.elasticsearch.xpack.transform.notifications;
 
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.indices.alias.Alias;
+import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
 import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.client.OriginSettingClient;
 import org.elasticsearch.cluster.metadata.AliasMetadata;
+import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
 import org.elasticsearch.cluster.metadata.IndexTemplateMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.xcontent.ToXContent;
@@ -23,6 +28,7 @@ import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformIn
 import org.elasticsearch.xpack.transform.persistence.TransformInternalIndex;
 
 import java.io.IOException;
+import java.util.Collections;
 
 import static org.elasticsearch.xpack.core.ClientHelper.TRANSFORM_ORIGIN;
 
@@ -34,22 +40,24 @@ public class TransformAuditor extends AbstractAuditor<TransformAuditMessage> {
     private volatile boolean isResetMode = false;
 
     public TransformAuditor(Client client, String nodeName, ClusterService clusterService) {
-        super(new OriginSettingClient(client, TRANSFORM_ORIGIN), TransformInternalIndexConstants.AUDIT_INDEX,
-            TransformInternalIndexConstants.AUDIT_INDEX, null,
+        super(
+            new OriginSettingClient(client, TRANSFORM_ORIGIN),
+            TransformInternalIndexConstants.AUDIT_INDEX,
+            TransformInternalIndexConstants.AUDIT_INDEX,
+            Version.V_7_16_0,
             () -> {
+                // legacy template implementation, to be removed in 8.x
                 try {
                     IndexTemplateMetadata templateMeta = TransformInternalIndex.getAuditIndexTemplateMetadata();
 
-                    PutIndexTemplateRequest request = new PutIndexTemplateRequest(templateMeta.name())
-                        .patterns(templateMeta.patterns())
+                    PutIndexTemplateRequest request = new PutIndexTemplateRequest(templateMeta.name()).patterns(templateMeta.patterns())
                         .version(templateMeta.version())
                         .settings(templateMeta.settings())
                         .mapping(templateMeta.mappings().uncompressed(), XContentType.JSON);
 
                     for (ObjectObjectCursor<String, AliasMetadata> cursor : templateMeta.getAliases()) {
                         AliasMetadata meta = cursor.value;
-                        Alias alias = new Alias(meta.alias())
-                            .indexRouting(meta.indexRouting())
+                        Alias alias = new Alias(meta.alias()).indexRouting(meta.indexRouting())
                             .searchRouting(meta.searchRouting())
                             .isHidden(meta.isHidden())
                             .writeIndex(meta.writeIndex());
@@ -62,10 +70,29 @@ public class TransformAuditor extends AbstractAuditor<TransformAuditMessage> {
 
                     return request;
                 } catch (IOException e) {
-                    return null;
+                    throw new ElasticsearchException("Failure creating transform notification index", e);
+                }
+            },
+            () -> {
+                try {
+                    PutComposableIndexTemplateAction.Request request = new PutComposableIndexTemplateAction.Request(
+                        TransformInternalIndexConstants.AUDIT_INDEX
+                    ).indexTemplate(
+                        new ComposableIndexTemplate.Builder().template(TransformInternalIndex.getAuditIndexTemplate())
+                            .version((long) Version.CURRENT.id)
+                            .indexPatterns(Collections.singletonList(TransformInternalIndexConstants.AUDIT_INDEX_PREFIX + "*"))
+                            .priority(Long.MAX_VALUE)
+                            .build()
+                    );
+                    return request;
+                } catch (IOException e) {
+                    throw new ElasticsearchException("Failure creating transform notification index", e);
                 }
             },
-            () -> null, nodeName, TransformAuditMessage::new, clusterService);
+            nodeName,
+            TransformAuditMessage::new,
+            clusterService
+        );
         clusterService.addListener(event -> {
             if (event.metadataChanged()) {
                 isResetMode = TransformMetadata.getTransformMetadata(event.state()).isResetMode();

+ 45 - 96
x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/TransformInternalIndex.java

@@ -14,24 +14,21 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
 import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
 import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
-import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest;
 import org.elasticsearch.action.support.ActiveShardCount;
-import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.AliasMetadata;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.IndexTemplateMetadata;
+import org.elasticsearch.cluster.metadata.Template;
 import org.elasticsearch.cluster.routing.IndexRoutingTable;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentHelper;
-import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.indices.SystemIndexDescriptor;
+import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xpack.core.common.notifications.AbstractAuditMessage;
 import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
 import org.elasticsearch.xpack.core.transform.TransformField;
@@ -48,8 +45,8 @@ import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformIn
 import java.io.IOException;
 import java.util.Collections;
 
-import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME;
+import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.xpack.core.ClientHelper.TRANSFORM_ORIGIN;
 import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
 import static org.elasticsearch.xpack.core.transform.TransformField.TRANSFORM_ID;
@@ -104,6 +101,8 @@ public final class TransformInternalIndex {
             .build();
     }
 
+    // use getAuditIndexTemplate instead
+    @Deprecated
     public static IndexTemplateMetadata getAuditIndexTemplateMetadata() throws IOException {
         IndexTemplateMetadata transformTemplate = IndexTemplateMetadata.builder(TransformInternalIndexConstants.AUDIT_INDEX)
             .patterns(Collections.singletonList(TransformInternalIndexConstants.AUDIT_INDEX_PREFIX + "*"))
@@ -121,6 +120,21 @@ public final class TransformInternalIndex {
         return transformTemplate;
     }
 
+    public static Template getAuditIndexTemplate() throws IOException {
+        AliasMetadata alias = AliasMetadata.builder(TransformInternalIndexConstants.AUDIT_INDEX_READ_ALIAS).isHidden(true).build();
+
+        return new Template(
+            Settings.builder()
+                // the audits are expected to be small
+                .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+                .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1")
+                .put(IndexMetadata.SETTING_INDEX_HIDDEN, true)
+                .build(),
+            new CompressedXContent(Strings.toString(auditMappings())),
+            Collections.singletonMap(alias.alias(), alias)
+        );
+    }
+
     private static XContentBuilder auditMappings() throws IOException {
         XContentBuilder builder = jsonBuilder().startObject();
         builder.startObject(SINGLE_MAPPING_NAME);
@@ -365,33 +379,6 @@ public final class TransformInternalIndex {
         return builder.startObject("_meta").field("version", Version.CURRENT).endObject();
     }
 
-    /**
-     * This method should be called before any document is indexed that relies on the
-     * existence of the latest internal index or audit index template.
-     *
-     * For audit messages the problem is that the standard template upgrader only runs
-     * when the master node is upgraded to the newer version.  So for the audit index,
-     * and also the internal index in the case where the old version doesn't know about
-     * system indices, if data nodes are upgraded before master nodes and transforms
-     * get assigned to those data nodes then without this check the data nodes will
-     * index documents into the internal index before the necessary index template is
-     * present and this will result in an index with completely dynamic mappings being
-     * created (which is very bad).  For the internal index in the case where the old
-     * version knows about system indices, if data nodes are upgraded before master
-     * nodes and transforms get assigned to those data nodes then without this check
-     * the data nodes will index documents into the internal index before the latest
-     * system descriptor is present on the master and this will result in an index with
-     * the new name but the old mappings.
-     */
-    public static void ensureLatestIndexAndTemplateInstalled(ClusterService clusterService, Client client, ActionListener<Void> listener) {
-
-        createLatestVersionedIndexIfRequired(
-            clusterService,
-            client,
-            ActionListener.wrap(r -> installLatestAuditIndexTemplateIfRequired(clusterService, client, listener), listener::onFailure)
-        );
-    }
-
     protected static boolean hasLatestVersionedIndex(ClusterState state) {
         return state.getMetadata().getIndicesLookup().containsKey(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME);
     }
@@ -402,10 +389,6 @@ public final class TransformInternalIndex {
         return indexRouting != null && indexRouting.allPrimaryShardsActive();
     }
 
-    protected static boolean hasLatestAuditIndexTemplate(ClusterState state) {
-        return state.getMetadata().getTemplates().containsKey(TransformInternalIndexConstants.AUDIT_INDEX);
-    }
-
     private static void waitForLatestVersionedIndexShardsActive(Client client, ActionListener<Void> listener) {
         ClusterHealthRequest request = new ClusterHealthRequest(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME)
             // cluster health does not wait for active shards per default
@@ -420,11 +403,14 @@ public final class TransformInternalIndex {
         );
     }
 
-    protected static void createLatestVersionedIndexIfRequired(
-        ClusterService clusterService,
-        Client client,
-        ActionListener<Void> listener
-    ) {
+    /**
+     * This method should be called before any document is indexed that relies on the
+     * existence of the latest internal index.
+     *
+     * Without this check the data nodes will create an internal index with dynamic
+     * mappings when indexing a document, but we want our own well defined mappings.
+     */
+    public static void createLatestVersionedIndexIfRequired(ClusterService clusterService, Client client, ActionListener<Void> listener) {
         ClusterState state = clusterService.state();
         // The check for existence is against local cluster state, so very cheap
         if (hasLatestVersionedIndex(state)) {
@@ -439,30 +425,28 @@ public final class TransformInternalIndex {
 
         // Creating the index involves communication with the master node, so it's more expensive but much rarer
         try {
-            CreateIndexRequest request = new CreateIndexRequest(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME)
-                .settings(settings())
+            CreateIndexRequest request = new CreateIndexRequest(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME).settings(
+                settings()
+            )
                 .mapping(mappings())
                 .origin(TRANSFORM_ORIGIN)
                 // explicitly wait for the primary shard (although this might be default)
                 .waitForActiveShards(ActiveShardCount.ONE);
-            ActionListener<CreateIndexResponse> innerListener = ActionListener.wrap(
-                r -> listener.onResponse(null),
-                e -> {
-                    // It's not a problem if the index already exists - another node could be running
-                    // this method at the same time as this one, and also have created the index
-                    // check if shards are active
-                    if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) {
-                        if (allPrimaryShardsActiveForLatestVersionedIndex(clusterService.state())) {
-                            listener.onResponse(null);
-                            return;
-                        }
-                        // the index exists but is not ready yet
-                        waitForLatestVersionedIndexShardsActive(client, listener);
-                    } else {
-                        listener.onFailure(e);
+            ActionListener<CreateIndexResponse> innerListener = ActionListener.wrap(r -> listener.onResponse(null), e -> {
+                // It's not a problem if the index already exists - another node could be running
+                // this method at the same time as this one, and also have created the index
+                // check if shards are active
+                if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) {
+                    if (allPrimaryShardsActiveForLatestVersionedIndex(clusterService.state())) {
+                        listener.onResponse(null);
+                        return;
                     }
+                    // the index exists but is not ready yet
+                    waitForLatestVersionedIndexShardsActive(client, listener);
+                } else {
+                    listener.onFailure(e);
                 }
-            );
+            });
             executeAsyncWithOrigin(
                 client.threadPool().getThreadContext(),
                 TRANSFORM_ORIGIN,
@@ -475,40 +459,5 @@ public final class TransformInternalIndex {
         }
     }
 
-    protected static void installLatestAuditIndexTemplateIfRequired(
-        ClusterService clusterService,
-        Client client,
-        ActionListener<Void> listener
-    ) {
-
-        // The check for existence of the template is against local cluster state, so very cheap
-        if (hasLatestAuditIndexTemplate(clusterService.state())) {
-            listener.onResponse(null);
-            return;
-        }
-
-        // Installing the template involves communication with the master node, so it's more expensive but much rarer
-        try {
-            IndexTemplateMetadata indexTemplateMetadata = getAuditIndexTemplateMetadata();
-            BytesReference jsonMappings = indexTemplateMetadata.mappings().uncompressed();
-            PutIndexTemplateRequest request = new PutIndexTemplateRequest(TransformInternalIndexConstants.AUDIT_INDEX).patterns(
-                indexTemplateMetadata.patterns()
-            )
-                .version(indexTemplateMetadata.version())
-                .settings(indexTemplateMetadata.settings())
-                .mapping(XContentHelper.convertToMap(jsonMappings, true, XContentType.JSON).v2());
-            ActionListener<AcknowledgedResponse> innerListener = ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure);
-            executeAsyncWithOrigin(
-                client.threadPool().getThreadContext(),
-                TRANSFORM_ORIGIN,
-                request,
-                innerListener,
-                client.admin().indices()::putTemplate
-            );
-        } catch (IOException e) {
-            listener.onFailure(e);
-        }
-    }
-
     private TransformInternalIndex() {}
 }

+ 2 - 2
x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformPersistentTasksExecutor.java

@@ -302,8 +302,8 @@ public class TransformPersistentTasksExecutor extends PersistentTasksExecutor<Tr
             }
         );
 
-        // <1> Check the index templates are installed
-        TransformInternalIndex.ensureLatestIndexAndTemplateInstalled(
+        // <1> Check the latest internal index (IMPORTANT: according to _this_ node, which might be newer than master) is installed
+        TransformInternalIndex.createLatestVersionedIndexIfRequired(
             clusterService,
             buildTask.getParentTaskClient(),
             templateCheckListener

+ 4 - 74
x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/persistence/TransformInternalIndexTests.java

@@ -12,7 +12,6 @@ import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
 import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
-import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.AdminClient;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.client.ClusterAdminClient;
@@ -53,7 +52,6 @@ import static org.mockito.Mockito.when;
 public class TransformInternalIndexTests extends ESTestCase {
 
     private ClusterState stateWithLatestVersionedIndex;
-    private ClusterState stateWithLatestAuditIndexTemplate;
 
     public static ClusterState randomTransformClusterState() {
         return randomTransformClusterState(true);
@@ -118,7 +116,6 @@ public class TransformInternalIndexTests extends ESTestCase {
     @Before
     public void setupClusterStates() {
         stateWithLatestVersionedIndex = randomTransformClusterState();
-        stateWithLatestAuditIndexTemplate = randomTransformAuditClusterState();
     }
 
     public void testHaveLatestVersionedIndexTemplate() {
@@ -307,67 +304,7 @@ public class TransformInternalIndexTests extends ESTestCase {
         verifyNoMoreInteractions(clusterClient);
     }
 
-    public void testHaveLatestAuditIndexTemplate() {
-
-        assertTrue(TransformInternalIndex.hasLatestAuditIndexTemplate(stateWithLatestAuditIndexTemplate));
-        assertFalse(TransformInternalIndex.hasLatestAuditIndexTemplate(ClusterState.EMPTY_STATE));
-    }
-
-    public void testInstallLatestAuditIndexTemplateIfRequired_GivenNotRequired() {
-
-        ClusterService clusterService = mock(ClusterService.class);
-        when(clusterService.state()).thenReturn(stateWithLatestAuditIndexTemplate);
-
-        Client client = mock(Client.class);
-
-        AtomicBoolean gotResponse = new AtomicBoolean(false);
-        ActionListener<Void> testListener = ActionListener.wrap(aVoid -> gotResponse.set(true), e -> fail(e.getMessage()));
-
-        TransformInternalIndex.installLatestAuditIndexTemplateIfRequired(clusterService, client, testListener);
-
-        assertTrue(gotResponse.get());
-        verifyNoMoreInteractions(client);
-    }
-
-    public void testInstallLatestAuditIndexTemplateIfRequired_GivenRequired() {
-
-        ClusterService clusterService = mock(ClusterService.class);
-        when(clusterService.state()).thenReturn(ClusterState.EMPTY_STATE);
-
-        IndicesAdminClient indicesClient = mock(IndicesAdminClient.class);
-        doAnswer(invocationOnMock -> {
-            @SuppressWarnings("unchecked")
-            ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
-            listener.onResponse(AcknowledgedResponse.TRUE);
-            return null;
-        }).when(indicesClient).putTemplate(any(), any());
-
-        AdminClient adminClient = mock(AdminClient.class);
-        when(adminClient.indices()).thenReturn(indicesClient);
-        Client client = mock(Client.class);
-        when(client.admin()).thenReturn(adminClient);
-
-        ThreadPool threadPool = mock(ThreadPool.class);
-        when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
-        when(client.threadPool()).thenReturn(threadPool);
-
-        AtomicBoolean gotResponse = new AtomicBoolean(false);
-        ActionListener<Void> testListener = ActionListener.wrap(aVoid -> gotResponse.set(true), e -> fail(e.getMessage()));
-
-        TransformInternalIndex.installLatestAuditIndexTemplateIfRequired(clusterService, client, testListener);
-
-        assertTrue(gotResponse.get());
-        verify(client, times(1)).threadPool();
-        verify(client, times(1)).admin();
-        verifyNoMoreInteractions(client);
-        verify(adminClient, times(1)).indices();
-        verifyNoMoreInteractions(adminClient);
-        verify(indicesClient, times(1)).putTemplate(any(), any());
-        verifyNoMoreInteractions(indicesClient);
-    }
-
     public void testEnsureLatestIndexAndTemplateInstalled_GivenRequired() {
-
         ClusterService clusterService = mock(ClusterService.class);
         when(clusterService.state()).thenReturn(ClusterState.EMPTY_STATE);
 
@@ -378,12 +315,6 @@ public class TransformInternalIndexTests extends ESTestCase {
             listener.onResponse(new CreateIndexResponse(true, true, TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME));
             return null;
         }).when(indicesClient).create(any(), any());
-        doAnswer(invocationOnMock -> {
-            @SuppressWarnings("unchecked")
-            ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
-            listener.onResponse(AcknowledgedResponse.TRUE);
-            return null;
-        }).when(indicesClient).putTemplate(any(), any());
 
         AdminClient adminClient = mock(AdminClient.class);
         when(adminClient.indices()).thenReturn(indicesClient);
@@ -397,16 +328,15 @@ public class TransformInternalIndexTests extends ESTestCase {
         AtomicBoolean gotResponse = new AtomicBoolean(false);
         ActionListener<Void> testListener = ActionListener.wrap(aVoid -> gotResponse.set(true), e -> fail(e.getMessage()));
 
-        TransformInternalIndex.ensureLatestIndexAndTemplateInstalled(clusterService, client, testListener);
+        TransformInternalIndex.createLatestVersionedIndexIfRequired(clusterService, client, testListener);
 
         assertTrue(gotResponse.get());
-        verify(client, times(2)).threadPool();
-        verify(client, times(2)).admin();
+        verify(client, times(1)).threadPool();
+        verify(client, times(1)).admin();
         verifyNoMoreInteractions(client);
-        verify(adminClient, times(2)).indices();
+        verify(adminClient, times(1)).indices();
         verifyNoMoreInteractions(adminClient);
         verify(indicesClient, times(1)).create(any(), any());
-        verify(indicesClient, times(1)).putTemplate(any(), any());
         verifyNoMoreInteractions(indicesClient);
     }
 }