Browse Source

Operator/index templates (#90143)

Adds support for /_index_template/ and 
/_component_template/ in file based settings.

Co-authored-by: Andrei Dan <andrei.dan@elastic.co>
Nikola Grcevski 3 years ago
parent
commit
9b0363ea2c

+ 5 - 0
docs/changelog/90143.yaml

@@ -0,0 +1,5 @@
+pr: 90143
+summary: Operator/index templates
+area: Infra/Core
+type: enhancement
+issues: []

+ 727 - 0
server/src/internalClusterTest/java/org/elasticsearch/reservedstate/service/ComponentTemplatesFileSettingsIT.java

@@ -0,0 +1,727 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.reservedstate.service;
+
+import org.elasticsearch.action.admin.indices.template.get.GetComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.get.GetComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.PutComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.ClusterChangedEvent;
+import org.elasticsearch.cluster.ClusterStateListener;
+import org.elasticsearch.cluster.metadata.ComponentTemplate;
+import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
+import org.elasticsearch.cluster.metadata.ReservedStateErrorMetadata;
+import org.elasticsearch.cluster.metadata.ReservedStateHandlerMetadata;
+import org.elasticsearch.cluster.metadata.ReservedStateMetadata;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.Strings;
+import org.elasticsearch.core.Tuple;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+
+import java.io.ByteArrayInputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction.reservedComposableIndexName;
+import static org.elasticsearch.test.NodeRoles.dataOnlyNode;
+import static org.elasticsearch.xcontent.XContentType.JSON;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.notNullValue;
+
+@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false)
+public class ComponentTemplatesFileSettingsIT extends ESIntegTestCase {
+
+    private static AtomicLong versionCounter = new AtomicLong(1);
+
+    private static String emptyJSON = """
+        {
+             "metadata": {
+                 "version": "%s",
+                 "compatibility": "8.4.0"
+             },
+             "state": {
+                "index_templates": {
+                }
+             }
+        }""";
+
+    private static String testJSON = """
+        {
+             "metadata": {
+                 "version": "%s",
+                 "compatibility": "8.4.0"
+             },
+             "state": {
+               "index_templates": {
+                 "component_templates": {
+                    "component_template1": {
+                      "template": {
+                        "mappings": {
+                          "properties": {
+                            "@timestamp": {
+                              "type": "date"
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "runtime_component_template": {
+                      "template": {
+                        "mappings": {
+                          "runtime": {
+                            "day_of_week": {
+                              "type": "keyword"
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "other_component_template": {
+                      "template": {
+                        "mappings": {
+                          "runtime": {
+                            "day_of_week": {
+                              "type": "keyword"
+                            }
+                          }
+                        }
+                      }
+                    }
+                 },
+                 "composable_index_templates": {
+                    "template_1": {
+                        "index_patterns": ["te*", "bar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 500,
+                        "composed_of": ["component_template1", "runtime_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    },
+                    "template_2": {
+                        "index_patterns": ["foo*", "mar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 100,
+                        "composed_of": ["component_template1", "runtime_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    },
+                    "template_other": {
+                        "index_patterns": ["foo*", "mar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 50,
+                        "composed_of": ["other_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    }
+                 }
+               }
+             }
+        }""";
+
+    private static String testJSONLess = """
+        {
+             "metadata": {
+                 "version": "%s",
+                 "compatibility": "8.4.0"
+             },
+             "state": {
+               "index_templates": {
+                 "component_templates": {
+                    "component_template1": {
+                      "template": {
+                        "mappings": {
+                          "properties": {
+                            "@timestamp": {
+                              "type": "date"
+                            }
+                          }
+                        }
+                      }
+                    },
+                    "runtime_component_template": {
+                      "template": {
+                        "mappings": {
+                          "runtime": {
+                            "day_of_week": {
+                              "type": "keyword"
+                            }
+                          }
+                        }
+                      }
+                    }
+                 },
+                 "composable_index_templates": {
+                    "template_1": {
+                        "index_patterns": ["te*", "bar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 500,
+                        "composed_of": ["component_template1", "runtime_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    },
+                    "template_2": {
+                        "index_patterns": ["foo*", "mar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 100,
+                        "composed_of": ["component_template1", "runtime_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    }
+                 }
+               }
+             }
+        }""";
+
+    private static String testErrorJSON = """
+        {
+             "metadata": {
+                 "version": "%s",
+                 "compatibility": "8.4.0"
+             },
+             "state": {
+               "index_templates": {
+                 "composable_index_templates": {
+                    "err_template": {
+                        "index_patterns": ["te*", "bar*"],
+                        "template": {
+                          "settings": {
+                            "number_of_shards": 1
+                          },
+                          "mappings": {
+                            "_source": {
+                              "enabled": true
+                            },
+                            "properties": {
+                              "host_name": {
+                                "type": "keyword"
+                              },
+                              "created_at": {
+                                "type": "date",
+                                "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                              }
+                            }
+                          },
+                          "aliases": {
+                            "mydata": { }
+                          }
+                        },
+                        "priority": 500,
+                        "composed_of": ["component_template1", "runtime_component_template"],
+                        "version": 3,
+                        "_meta": {
+                          "description": "my custom"
+                        }
+                    }
+                 }
+               }
+             }
+        }""";
+
+    private void assertMasterNode(Client client, String node) throws ExecutionException, InterruptedException {
+        assertThat(client.admin().cluster().prepareState().execute().get().getState().nodes().getMasterNode().getName(), equalTo(node));
+    }
+
+    private void writeJSONFile(String node, String json) throws Exception {
+        long version = versionCounter.incrementAndGet();
+
+        FileSettingsService fileSettingsService = internalCluster().getInstance(FileSettingsService.class, node);
+
+        Files.createDirectories(fileSettingsService.operatorSettingsDir());
+        Path tempFilePath = createTempFile();
+
+        Files.write(tempFilePath, Strings.format(json, version).getBytes(StandardCharsets.UTF_8));
+        Files.move(tempFilePath, fileSettingsService.operatorSettingsFile(), StandardCopyOption.ATOMIC_MOVE);
+    }
+
+    private Tuple<CountDownLatch, AtomicLong> setupClusterStateListener(String node) {
+        ClusterService clusterService = internalCluster().clusterService(node);
+        CountDownLatch savedClusterState = new CountDownLatch(1);
+        AtomicLong metadataVersion = new AtomicLong(-1);
+        clusterService.addListener(new ClusterStateListener() {
+            @Override
+            public void clusterChanged(ClusterChangedEvent event) {
+                ReservedStateMetadata reservedState = event.state().metadata().reservedStateMetadata().get(FileSettingsService.NAMESPACE);
+                if (reservedState != null) {
+                    ReservedStateHandlerMetadata handlerMetadata = reservedState.handlers().get(ReservedComposableIndexTemplateAction.NAME);
+                    if (handlerMetadata != null && handlerMetadata.keys().contains(reservedComposableIndexName("template_1"))) {
+                        clusterService.removeListener(this);
+                        metadataVersion.set(event.state().metadata().version());
+                        savedClusterState.countDown();
+                    } else if (reservedState.errorMetadata() != null) {
+                        clusterService.removeListener(this);
+                        savedClusterState.countDown();
+                        throw new IllegalStateException(String.join(",", reservedState.errorMetadata().errors()));
+                    }
+                }
+            }
+        });
+
+        return new Tuple<>(savedClusterState, metadataVersion);
+    }
+
+    private void assertClusterStateSaveOK(CountDownLatch savedClusterState, AtomicLong metadataVersion) throws Exception {
+        boolean awaitSuccessful = savedClusterState.await(20, TimeUnit.SECONDS);
+        assertTrue(awaitSuccessful);
+
+        final var response = client().execute(
+            GetComposableIndexTemplateAction.INSTANCE,
+            new GetComposableIndexTemplateAction.Request("template*")
+        ).get();
+
+        assertThat(
+            response.indexTemplates().keySet().stream().collect(Collectors.toSet()),
+            containsInAnyOrder("template_1", "template_2", "template_other")
+        );
+
+        assertTrue(
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> client().execute(PutComponentTemplateAction.INSTANCE, sampleComponentRestRequest("component_template1")).actionGet()
+            ).getMessage().contains("[[component_template:component_template1] set as read-only by [file_settings]]")
+        );
+
+        assertTrue(
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> client().execute(PutComposableIndexTemplateAction.INSTANCE, sampleIndexTemplateRestRequest("template_1")).actionGet()
+            ).getMessage().contains("[[composable_index_template:template_1] set as read-only by [file_settings]]")
+        );
+    }
+
+    private Tuple<CountDownLatch, AtomicLong> setupClusterStateListenerForOtherDelete(String node) {
+        ClusterService clusterService = internalCluster().clusterService(node);
+        CountDownLatch savedClusterState = new CountDownLatch(1);
+        AtomicLong metadataVersion = new AtomicLong(-1);
+        clusterService.addListener(new ClusterStateListener() {
+            @Override
+            public void clusterChanged(ClusterChangedEvent event) {
+                ReservedStateMetadata reservedState = event.state().metadata().reservedStateMetadata().get(FileSettingsService.NAMESPACE);
+                if (reservedState != null) {
+                    ReservedStateHandlerMetadata handlerMetadata = reservedState.handlers().get(ReservedComposableIndexTemplateAction.NAME);
+                    if (handlerMetadata != null
+                        && handlerMetadata.keys().isEmpty() == false
+                        && handlerMetadata.keys().contains(reservedComposableIndexName("template_other")) == false) {
+                        clusterService.removeListener(this);
+                        metadataVersion.set(event.state().metadata().version());
+                        savedClusterState.countDown();
+                    } else if (reservedState.errorMetadata() != null) {
+                        clusterService.removeListener(this);
+                        savedClusterState.countDown();
+                        throw new IllegalStateException(String.join(",", reservedState.errorMetadata().errors()));
+                    }
+                }
+            }
+        });
+
+        return new Tuple<>(savedClusterState, metadataVersion);
+    }
+
+    private void assertComponentAndIndexTemplateDelete(CountDownLatch savedClusterState, AtomicLong metadataVersion) throws Exception {
+        boolean awaitSuccessful = savedClusterState.await(20, TimeUnit.SECONDS);
+        assertTrue(awaitSuccessful);
+
+        final var response = client().execute(
+            GetComposableIndexTemplateAction.INSTANCE,
+            new GetComposableIndexTemplateAction.Request("template*")
+        ).get();
+
+        assertThat(response.indexTemplates().keySet().stream().collect(Collectors.toSet()), containsInAnyOrder("template_1", "template_2"));
+
+        final var componentResponse = client().execute(
+            GetComponentTemplateAction.INSTANCE,
+            new GetComponentTemplateAction.Request("other*")
+        ).get();
+
+        assertTrue(componentResponse.getComponentTemplates().isEmpty());
+
+        // this should just work, other is not locked
+        client().execute(PutComponentTemplateAction.INSTANCE, sampleComponentRestRequest("other_component_template")).get();
+
+        // this will fail now because sampleIndexTemplateRestRequest wants use the component templates
+        // ["component_template1", "runtime_component_template"], which are not allowed to be used by REST requests, since they
+        // are written by file based settings, e.g. in operator mode. Allowing REST requests to use these components would mean that
+        // we would be unable to delete these components with file based settings, since they would be used by various composable
+        // index templates not managed by file based settings.
+        assertTrue(
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> client().execute(PutComposableIndexTemplateAction.INSTANCE, sampleIndexTemplateRestRequest("template_other"))
+                    .actionGet()
+            ).getMessage()
+                .contains(
+                    "with errors: [[component_template:runtime_component_template, "
+                        + "component_template:component_template1] is reserved by [file_settings]]"
+                )
+        );
+
+        // this will work now, we are saving template without components
+        client().execute(PutComposableIndexTemplateAction.INSTANCE, sampleIndexTemplateRestRequestNoComponents("template_other")).get();
+
+        // the rest are still locked
+        assertTrue(
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> client().execute(PutComponentTemplateAction.INSTANCE, sampleComponentRestRequest("component_template1")).actionGet()
+            ).getMessage().contains("[[component_template:component_template1] set as read-only by [file_settings]]")
+        );
+
+        assertTrue(
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> client().execute(PutComposableIndexTemplateAction.INSTANCE, sampleIndexTemplateRestRequest("template_1")).actionGet()
+            ).getMessage().contains("[[composable_index_template:template_1] set as read-only by [file_settings]]")
+        );
+    }
+
+    private Tuple<CountDownLatch, AtomicLong> setupClusterStateListenerForCleanup(String node) {
+        ClusterService clusterService = internalCluster().clusterService(node);
+        CountDownLatch savedClusterState = new CountDownLatch(1);
+        AtomicLong metadataVersion = new AtomicLong(-1);
+        clusterService.addListener(new ClusterStateListener() {
+            @Override
+            public void clusterChanged(ClusterChangedEvent event) {
+                ReservedStateMetadata reservedState = event.state().metadata().reservedStateMetadata().get(FileSettingsService.NAMESPACE);
+                if (reservedState != null) {
+                    ReservedStateHandlerMetadata handlerMetadata = reservedState.handlers().get(ReservedComposableIndexTemplateAction.NAME);
+                    if (handlerMetadata == null || handlerMetadata.keys().isEmpty()) {
+                        clusterService.removeListener(this);
+                        metadataVersion.set(event.state().metadata().version());
+                        savedClusterState.countDown();
+                    }
+                }
+            }
+        });
+
+        return new Tuple<>(savedClusterState, metadataVersion);
+    }
+
+    public void testSettingsApplied() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        logger.info("--> start data node / non master node");
+        var dataNode = internalCluster().startNode(Settings.builder().put(dataOnlyNode()).put("discovery.initial_state_timeout", "1s"));
+
+        var savedClusterState = setupClusterStateListener(dataNode);
+        // In internal cluster tests, the nodes share the config directory, so when we write with the data node path
+        // the master will pick it up on start
+        logger.info("--> write the initial settings json with all component templates and composable index templates");
+        writeJSONFile(dataNode, testJSON);
+
+        logger.info("--> start master node");
+        final String masterNode = internalCluster().startMasterOnlyNode();
+        assertMasterNode(internalCluster().nonMasterClient(), masterNode);
+
+        assertClusterStateSaveOK(savedClusterState.v1(), savedClusterState.v2());
+
+        savedClusterState = setupClusterStateListenerForOtherDelete(internalCluster().getMasterName());
+        logger.info("--> write the reduced JSON, so we delete template_other and other_component_template");
+        writeJSONFile(internalCluster().getMasterName(), testJSONLess);
+
+        assertComponentAndIndexTemplateDelete(savedClusterState.v1(), savedClusterState.v2());
+
+        logger.info("---> cleanup file based settings...");
+        // if clean-up doesn't succeed correctly, TestCluster.wipeAllComposableIndexTemplates will fail
+        savedClusterState = setupClusterStateListenerForCleanup(internalCluster().getMasterName());
+        writeJSONFile(internalCluster().getMasterName(), emptyJSON);
+        boolean awaitSuccessful = savedClusterState.v1().await(20, TimeUnit.SECONDS);
+        assertTrue(awaitSuccessful);
+    }
+
+    private Tuple<CountDownLatch, AtomicLong> setupClusterStateListenerForError(String node) {
+        ClusterService clusterService = internalCluster().clusterService(node);
+        CountDownLatch savedClusterState = new CountDownLatch(1);
+        AtomicLong metadataVersion = new AtomicLong(-1);
+        clusterService.addListener(new ClusterStateListener() {
+            @Override
+            public void clusterChanged(ClusterChangedEvent event) {
+                ReservedStateMetadata reservedState = event.state().metadata().reservedStateMetadata().get(FileSettingsService.NAMESPACE);
+                if (reservedState != null && reservedState.errorMetadata() != null) {
+                    assertEquals(ReservedStateErrorMetadata.ErrorKind.VALIDATION, reservedState.errorMetadata().errorKind());
+                    assertThat(reservedState.errorMetadata().errors(), allOf(notNullValue(), hasSize(1)));
+                    assertThat(
+                        reservedState.errorMetadata().errors().get(0),
+                        containsString(
+                            "index_template [err_template] invalid, cause [index template [err_template] specifies "
+                                + "component templates [component_template1, runtime_component_template] that do not exist]"
+                        )
+                    );
+                    clusterService.removeListener(this);
+                    metadataVersion.set(event.state().metadata().version());
+                    savedClusterState.countDown();
+                }
+            }
+        });
+
+        return new Tuple<>(savedClusterState, metadataVersion);
+    }
+
+    private void assertClusterStateNotSaved(CountDownLatch savedClusterState, AtomicLong metadataVersion) throws Exception {
+        boolean awaitSuccessful = savedClusterState.await(20, TimeUnit.SECONDS);
+        assertTrue(awaitSuccessful);
+
+        final var response = client().execute(
+            GetComposableIndexTemplateAction.INSTANCE,
+            new GetComposableIndexTemplateAction.Request("err*")
+        ).get();
+
+        assertTrue(response.indexTemplates().isEmpty());
+
+        // This should succeed, nothing was reserved
+        client().execute(PutComposableIndexTemplateAction.INSTANCE, sampleIndexTemplateRestRequestNoComponents("err_template")).get();
+    }
+
+    public void testErrorSaved() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        logger.info("--> start data node / non master node");
+        internalCluster().startNode(Settings.builder().put(dataOnlyNode()).put("discovery.initial_state_timeout", "1s"));
+
+        logger.info("--> start master node");
+        final String masterNode = internalCluster().startMasterOnlyNode();
+        assertMasterNode(internalCluster().nonMasterClient(), masterNode);
+        var savedClusterState = setupClusterStateListenerForError(masterNode);
+
+        writeJSONFile(masterNode, testErrorJSON);
+        assertClusterStateNotSaved(savedClusterState.v1(), savedClusterState.v2());
+    }
+
+    private PutComponentTemplateAction.Request sampleComponentRestRequest(String name) throws Exception {
+        var json = """
+            {
+              "template": {
+                "mappings": {
+                  "properties": {
+                    "@timestamp": {
+                      "type": "date"
+                    }
+                  }
+                }
+              }
+            }""";
+
+        try (
+            var bis = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8));
+            var parser = JSON.xContent().createParser(XContentParserConfiguration.EMPTY, bis)
+        ) {
+            return new PutComponentTemplateAction.Request(name).componentTemplate(ComponentTemplate.parse(parser));
+        }
+    }
+
+    private PutComposableIndexTemplateAction.Request sampleIndexTemplateRestRequest(String name) throws Exception {
+        var json = """
+            {
+                "index_patterns": ["te*", "bar*"],
+                "template": {
+                  "settings": {
+                    "number_of_shards": 1
+                  },
+                  "mappings": {
+                    "_source": {
+                      "enabled": true
+                    },
+                    "properties": {
+                      "host_name": {
+                        "type": "keyword"
+                      },
+                      "created_at": {
+                        "type": "date",
+                        "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                      }
+                    }
+                  },
+                  "aliases": {
+                    "mydata": { }
+                  }
+                },
+                "priority": 500,
+                "composed_of": ["component_template1", "runtime_component_template"],
+                "version": 3,
+                "_meta": {
+                  "description": "my custom"
+                }
+            }""";
+
+        try (
+            var bis = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8));
+            var parser = JSON.xContent().createParser(XContentParserConfiguration.EMPTY, bis)
+        ) {
+            return new PutComposableIndexTemplateAction.Request(name).indexTemplate(ComposableIndexTemplate.parse(parser));
+        }
+    }
+
+    private PutComposableIndexTemplateAction.Request sampleIndexTemplateRestRequestNoComponents(String name) throws Exception {
+        var json = """
+            {
+                "index_patterns": ["aa*", "vv*"],
+                "template": {
+                  "settings": {
+                    "number_of_shards": 1
+                  },
+                  "mappings": {
+                    "_source": {
+                      "enabled": true
+                    },
+                    "properties": {
+                      "host_name": {
+                        "type": "keyword"
+                      },
+                      "created_at": {
+                        "type": "date",
+                        "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                      }
+                    }
+                  },
+                  "aliases": {
+                    "mydata": { }
+                  }
+                },
+                "priority": 500,
+                "version": 3,
+                "_meta": {
+                  "description": "my custom"
+                }
+            }""";
+
+        try (
+            var bis = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8));
+            var parser = JSON.xContent().createParser(XContentParserConfiguration.EMPTY, bis)
+        ) {
+            return new PutComposableIndexTemplateAction.Request(name).indexTemplate(ComposableIndexTemplate.parse(parser));
+        }
+    }
+
+}

+ 18 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteComponentTemplateAction.java

@@ -9,6 +9,7 @@
 package org.elasticsearch.action.admin.indices.template.delete;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
@@ -23,6 +24,11 @@ import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 public class TransportDeleteComponentTemplateAction extends AcknowledgedTransportMasterNodeAction<DeleteComponentTemplateAction.Request> {
 
     private final MetadataIndexTemplateService indexTemplateService;
@@ -63,4 +69,16 @@ public class TransportDeleteComponentTemplateAction extends AcknowledgedTranspor
     ) {
         indexTemplateService.removeComponentTemplate(request.names(), request.masterNodeTimeout(), state, listener);
     }
+
+    @Override
+    public Optional<String> reservedStateHandlerName() {
+        return Optional.of(ReservedComposableIndexTemplateAction.NAME);
+    }
+
+    @Override
+    public Set<String> modifiedKeys(DeleteComponentTemplateAction.Request request) {
+        return Arrays.stream(request.names())
+            .map(n -> ReservedComposableIndexTemplateAction.reservedComponentName(n))
+            .collect(Collectors.toSet());
+    }
 }

+ 18 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteComposableIndexTemplateAction.java

@@ -9,6 +9,7 @@
 package org.elasticsearch.action.admin.indices.template.delete;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
@@ -23,6 +24,11 @@ import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 public class TransportDeleteComposableIndexTemplateAction extends AcknowledgedTransportMasterNodeAction<
     DeleteComposableIndexTemplateAction.Request> {
 
@@ -64,4 +70,16 @@ public class TransportDeleteComposableIndexTemplateAction extends AcknowledgedTr
     ) {
         indexTemplateService.removeIndexTemplateV2(request.names(), request.masterNodeTimeout(), listener);
     }
+
+    @Override
+    public Optional<String> reservedStateHandlerName() {
+        return Optional.of(ReservedComposableIndexTemplateAction.NAME);
+    }
+
+    @Override
+    public Set<String> modifiedKeys(DeleteComposableIndexTemplateAction.Request request) {
+        return Arrays.stream(request.names())
+            .map(n -> ReservedComposableIndexTemplateAction.reservedComposableIndexName(n))
+            .collect(Collectors.toSet());
+    }
 }

+ 29 - 7
server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutComponentTemplateAction.java

@@ -9,6 +9,7 @@
 package org.elasticsearch.action.admin.indices.template.put;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
@@ -28,6 +29,9 @@ import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.Optional;
+import java.util.Set;
+
 public class TransportPutComponentTemplateAction extends AcknowledgedTransportMasterNodeAction<PutComponentTemplateAction.Request> {
 
     private final MetadataIndexTemplateService indexTemplateService;
@@ -62,14 +66,10 @@ public class TransportPutComponentTemplateAction extends AcknowledgedTransportMa
         return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
     }
 
-    @Override
-    protected void masterOperation(
-        Task task,
-        final PutComponentTemplateAction.Request request,
-        final ClusterState state,
-        final ActionListener<AcknowledgedResponse> listener
+    public static ComponentTemplate normalizeComponentTemplate(
+        ComponentTemplate componentTemplate,
+        IndexScopedSettings indexScopedSettings
     ) {
-        ComponentTemplate componentTemplate = request.componentTemplate();
         Template template = componentTemplate.template();
         // Normalize the index settings if necessary
         if (template.settings() != null) {
@@ -79,6 +79,18 @@ public class TransportPutComponentTemplateAction extends AcknowledgedTransportMa
             template = new Template(settings, template.mappings(), template.aliases());
             componentTemplate = new ComponentTemplate(template, componentTemplate.version(), componentTemplate.metadata());
         }
+
+        return componentTemplate;
+    }
+
+    @Override
+    protected void masterOperation(
+        Task task,
+        final PutComponentTemplateAction.Request request,
+        final ClusterState state,
+        final ActionListener<AcknowledgedResponse> listener
+    ) {
+        ComponentTemplate componentTemplate = normalizeComponentTemplate(request.componentTemplate(), indexScopedSettings);
         indexTemplateService.putComponentTemplate(
             request.cause(),
             request.create(),
@@ -88,4 +100,14 @@ public class TransportPutComponentTemplateAction extends AcknowledgedTransportMa
             listener
         );
     }
+
+    @Override
+    public Optional<String> reservedStateHandlerName() {
+        return Optional.of(ReservedComposableIndexTemplateAction.NAME);
+    }
+
+    @Override
+    public Set<String> modifiedKeys(PutComponentTemplateAction.Request request) {
+        return Set.of(ReservedComposableIndexTemplateAction.reservedComponentName(request.name()));
+    }
 }

+ 47 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutComposableIndexTemplateAction.java

@@ -9,6 +9,7 @@
 package org.elasticsearch.action.admin.indices.template.put;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
@@ -18,12 +19,21 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
+import org.elasticsearch.cluster.metadata.ReservedStateMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.core.Strings.format;
+
 public class TransportPutComposableIndexTemplateAction extends AcknowledgedTransportMasterNodeAction<
     PutComposableIndexTemplateAction.Request> {
 
@@ -63,6 +73,7 @@ public class TransportPutComposableIndexTemplateAction extends AcknowledgedTrans
         final ClusterState state,
         final ActionListener<AcknowledgedResponse> listener
     ) {
+        verifyIfUsingReservedComponentTemplates(request, state);
         ComposableIndexTemplate indexTemplate = request.indexTemplate();
         indexTemplateService.putIndexTemplateV2(
             request.cause(),
@@ -73,4 +84,40 @@ public class TransportPutComposableIndexTemplateAction extends AcknowledgedTrans
             listener
         );
     }
+
+    public static void verifyIfUsingReservedComponentTemplates(
+        final PutComposableIndexTemplateAction.Request request,
+        final ClusterState state
+    ) {
+        ComposableIndexTemplate indexTemplate = request.indexTemplate();
+        Set<String> composedOfKeys = indexTemplate.composedOf()
+            .stream()
+            .map(c -> ReservedComposableIndexTemplateAction.reservedComponentName(c))
+            .collect(Collectors.toSet());
+
+        List<String> errors = new ArrayList<>();
+
+        for (ReservedStateMetadata metadata : state.metadata().reservedStateMetadata().values()) {
+            Set<String> conflicts = metadata.conflicts(ReservedComposableIndexTemplateAction.NAME, composedOfKeys);
+            if (conflicts.isEmpty() == false) {
+                errors.add(format("[%s] is reserved by [%s]", String.join(", ", conflicts), metadata.namespace()));
+            }
+        }
+
+        if (errors.isEmpty() == false) {
+            throw new IllegalArgumentException(
+                format("Failed to process request [%s] with errors: [%s]", request, String.join(", ", errors))
+            );
+        }
+    }
+
+    @Override
+    public Optional<String> reservedStateHandlerName() {
+        return Optional.of(ReservedComposableIndexTemplateAction.NAME);
+    }
+
+    @Override
+    public Set<String> modifiedKeys(PutComposableIndexTemplateAction.Request request) {
+        return Set.of(ReservedComposableIndexTemplateAction.reservedComposableIndexName(request.name()));
+    }
 }

+ 243 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/template/reservedstate/ReservedComposableIndexTemplateAction.java

@@ -0,0 +1,243 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.action.admin.indices.template.reservedstate;
+
+import org.elasticsearch.action.admin.indices.template.put.PutComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.TransportPutComponentTemplateAction;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.ComponentTemplate;
+import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
+import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
+import org.elasticsearch.common.settings.IndexScopedSettings;
+import org.elasticsearch.common.util.set.Sets;
+import org.elasticsearch.reservedstate.ReservedClusterStateHandler;
+import org.elasticsearch.reservedstate.TransformState;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.common.xcontent.XContentHelper.mapToXContentParser;
+
+/**
+ * This {@link ReservedClusterStateHandler} is responsible for reserved state
+ * CRUD operations on composable index templates and component templates, e.g. file based settings.
+ * <p>
+ * Internally it uses {@link MetadataIndexTemplateService} to add, update and delete both composable
+ * index templates and component templates. The reserved state handler is implemented as a joint handler
+ * for both component templates and composable index templates, because of the inherent interdependencies
+ * of the two separate types. For example, one cannot create composable index templates without first the
+ * component definitions being in the cluster state, however, the opposite is true when deleting. This
+ * circular dependency makes it impossible for separation of the two handlers.
+ */
+public class ReservedComposableIndexTemplateAction
+    implements
+        ReservedClusterStateHandler<ReservedComposableIndexTemplateAction.ComponentsAndComposables> {
+    public static final String NAME = "index_templates";
+    public static final String COMPONENTS = "component_templates";
+    private static final String COMPONENT_PREFIX = "component_template:";
+    public static final String COMPOSABLES = "composable_index_templates";
+    public static final String COMPOSABLE_PREFIX = "composable_index_template:";
+
+    private final MetadataIndexTemplateService indexTemplateService;
+    private final IndexScopedSettings indexScopedSettings;
+
+    public ReservedComposableIndexTemplateAction(
+        MetadataIndexTemplateService indexTemplateService,
+        IndexScopedSettings indexScopedSettings
+    ) {
+        this.indexTemplateService = indexTemplateService;
+        this.indexScopedSettings = indexScopedSettings;
+    }
+
+    @Override
+    public String name() {
+        return NAME;
+    }
+
+    // Since we can't split the reserved state handler into two separate handlers, because of the
+    // circular dependency on create and delete, we must store both the component template keys and
+    // the composable index template keys in the same reserved state handler. To be able to correctly
+    // distinguish between the component names and the composable names, we prefix the reserved keys
+    // when they are stored in the cluster state. Similarly, we remove the prefix when we need to perform
+    // the REST API validation in the corresponding transport actions.
+
+    /**
+     * Prefixes the component template name with a prefix for storage in the cluster state
+     * @param name component template name
+     * @return prefixed component template name for storage in the reserved cluster state
+     */
+    public static String reservedComponentName(String name) {
+        return COMPONENT_PREFIX + name;
+    }
+
+    /**
+     * Removes the reserved cluster state prefix from the component template name
+     * <p>
+     * Once the prefix is removed we can use the name for conflict validation in {@link TransportPutComponentTemplateAction} and
+     * {@link org.elasticsearch.action.admin.indices.template.delete.TransportDeleteComponentTemplateAction}
+     * @param name the prefixed reserved component template name
+     * @return the un-prefixed component template name used for conflict validation at REST
+     */
+    public static String componentNameFromReservedName(String name) {
+        assert name.startsWith(COMPONENT_PREFIX);
+        return name.substring(COMPONENT_PREFIX.length());
+    }
+
+    /**
+     * Prefixes the composable index template name with a prefix for storage in the cluster state
+     * @param name composable index template name
+     * @return prefixed composable index template name for storage in the reserved cluster state
+     */
+    public static String reservedComposableIndexName(String name) {
+        return COMPOSABLE_PREFIX + name;
+    }
+
+    /**
+     * Removes the reserved cluster state prefix from the composable index template name
+     * <p>
+     * Once the prefix is removed we can use the name for conflict validation in
+     * {@link org.elasticsearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction} and
+     * {@link org.elasticsearch.action.admin.indices.template.delete.TransportDeleteComposableIndexTemplateAction}
+     * @param name the prefixed reserved composable index template name
+     * @return the un-prefixed composable index template name used for conflict validation at REST
+     */
+    public static String composableIndexNameFromReservedName(String name) {
+        assert name.startsWith(COMPOSABLE_PREFIX);
+        return name.substring(COMPOSABLE_PREFIX.length());
+    }
+
+    private ComponentsAndComposables prepare(ComponentsAndComposables componentsAndComposables) {
+        for (var request : componentsAndComposables.componentTemplates) {
+            validate(request);
+        }
+
+        for (var request : componentsAndComposables.composableTemplates) {
+            validate(request);
+        }
+
+        return componentsAndComposables;
+    }
+
+    @Override
+    public TransformState transform(Object source, TransformState prevState) throws Exception {
+        @SuppressWarnings("unchecked")
+        var requests = prepare((ComponentsAndComposables) source);
+        ClusterState state = prevState.state();
+
+        // We transform in the following order:
+        // 1. create or update component templates (composable templates depend on them)
+        // 2. create or update composable index templates (with disabled v2 overlap validation, we might delete some at step 3,
+        // while, 2 and 3 cannot be reversed because of data streams)
+        // 3. delete composable index templates (this will fail on attached data streams, unless we added higher priority one)
+        // 4. validate for v2 composable template overlaps
+        // 5. delete component templates (this will check if there are any related composable index templates and fail)
+
+        var components = requests.componentTemplates;
+        var composables = requests.composableTemplates;
+
+        // 1. create or update component templates (composable templates depend on them)
+        for (var request : components) {
+            ComponentTemplate template = TransportPutComponentTemplateAction.normalizeComponentTemplate(
+                request.componentTemplate(),
+                indexScopedSettings
+            );
+
+            state = indexTemplateService.addComponentTemplate(state, false, request.name(), template);
+        }
+
+        // 2. create or update composable index templates, no overlap validation
+        for (var request : composables) {
+            MetadataIndexTemplateService.validateV2TemplateRequest(state.metadata(), request.name(), request.indexTemplate());
+            state = indexTemplateService.addIndexTemplateV2(state, false, request.name(), request.indexTemplate(), false);
+        }
+
+        Set<String> composableEntities = composables.stream().map(r -> reservedComposableIndexName(r.name())).collect(Collectors.toSet());
+        Set<String> composablesToDelete = new HashSet<>(
+            prevState.keys().stream().filter(k -> k.startsWith(COMPOSABLE_PREFIX)).collect(Collectors.toSet())
+        );
+        composablesToDelete.removeAll(composableEntities);
+
+        // 3. delete composable index templates (this will fail on attached data streams, unless we added a higher priority one)
+        if (composablesToDelete.isEmpty() == false) {
+            var composableNames = composablesToDelete.stream().map(c -> composableIndexNameFromReservedName(c)).toArray(String[]::new);
+            state = MetadataIndexTemplateService.innerRemoveIndexTemplateV2(state, composableNames);
+        }
+
+        // 4. validate for v2 composable template overlaps
+        for (var request : composables) {
+            indexTemplateService.v2TemplateOverlaps(state, request.name(), request.indexTemplate(), true);
+        }
+
+        Set<String> componentEntities = components.stream().map(r -> reservedComponentName(r.name())).collect(Collectors.toSet());
+        Set<String> componentsToDelete = new HashSet<>(
+            prevState.keys().stream().filter(k -> k.startsWith(COMPONENT_PREFIX)).collect(Collectors.toSet())
+        );
+        componentsToDelete.removeAll(componentEntities);
+
+        // 5. delete component templates (this will check if there are any related composable index templates and fail)
+        if (componentsToDelete.isEmpty() == false) {
+            var componentNames = componentsToDelete.stream().map(c -> componentNameFromReservedName(c)).toArray(String[]::new);
+            state = MetadataIndexTemplateService.innerRemoveComponentTemplate(state, componentNames);
+        }
+
+        return new TransformState(state, Sets.union(componentEntities, composableEntities));
+    }
+
+    @Override
+    public ComponentsAndComposables fromXContent(XContentParser parser) throws IOException {
+        List<PutComponentTemplateAction.Request> componentTemplates = new ArrayList<>();
+        List<PutComposableIndexTemplateAction.Request> composableTemplates = new ArrayList<>();
+        Map<String, ?> source = parser.map();
+
+        @SuppressWarnings("unchecked")
+        Map<String, ?> components = (Map<String, ?>) source.get(COMPONENTS);
+
+        if (components != null) {
+            for (var entry : components.entrySet()) {
+                @SuppressWarnings("unchecked")
+                Map<String, ?> content = (Map<String, ?>) entry.getValue();
+                try (XContentParser componentParser = mapToXContentParser(XContentParserConfiguration.EMPTY, content)) {
+                    var componentTemplate = new PutComponentTemplateAction.Request(entry.getKey());
+                    componentTemplate.componentTemplate(ComponentTemplate.parse(componentParser));
+                    componentTemplates.add(componentTemplate);
+                }
+            }
+        }
+
+        @SuppressWarnings("unchecked")
+        Map<String, ?> composables = (Map<String, ?>) source.get(COMPOSABLES);
+
+        if (composables != null) {
+            for (var entry : composables.entrySet()) {
+                @SuppressWarnings("unchecked")
+                Map<String, ?> content = (Map<String, ?>) entry.getValue();
+                try (XContentParser componentParser = mapToXContentParser(XContentParserConfiguration.EMPTY, content)) {
+                    var composableTemplate = new PutComposableIndexTemplateAction.Request(entry.getKey());
+                    composableTemplate.indexTemplate(ComposableIndexTemplate.parse(componentParser));
+                    composableTemplates.add(composableTemplate);
+                }
+            }
+        }
+
+        return new ComponentsAndComposables(componentTemplates, composableTemplates);
+    }
+
+    record ComponentsAndComposables(
+        List<PutComponentTemplateAction.Request> componentTemplates,
+        List<PutComposableIndexTemplateAction.Request> composableTemplates
+    ) {}
+}

+ 58 - 25
server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java

@@ -393,7 +393,8 @@ public class MetadataIndexTemplateService {
         );
     }
 
-    static ClusterState innerRemoveComponentTemplate(ClusterState currentState, String... names) {
+    // Exposed for ReservedComponentTemplateAction
+    public static ClusterState innerRemoveComponentTemplate(ClusterState currentState, String... names) {
         validateNotInUse(currentState.metadata(), names);
 
         final Set<String> templateNames = new HashSet<>();
@@ -530,34 +531,23 @@ public class MetadataIndexTemplateService {
         final boolean create,
         final String name,
         final ComposableIndexTemplate template
+    ) throws Exception {
+        return addIndexTemplateV2(currentState, create, name, template, true);
+    }
+
+    public ClusterState addIndexTemplateV2(
+        final ClusterState currentState,
+        final boolean create,
+        final String name,
+        final ComposableIndexTemplate template,
+        final boolean validateV2Overlaps
     ) throws Exception {
         final ComposableIndexTemplate existing = currentState.metadata().templatesV2().get(name);
         if (create && existing != null) {
             throw new IllegalArgumentException("index template [" + name + "] already exists");
         }
 
-        Map<String, List<String>> overlaps = findConflictingV2Templates(
-            currentState,
-            name,
-            template.indexPatterns(),
-            true,
-            template.priorityOrZero()
-        );
-        overlaps.remove(name);
-        if (overlaps.size() > 0) {
-            String error = String.format(
-                Locale.ROOT,
-                "index template [%s] has index patterns %s matching patterns from "
-                    + "existing templates [%s] with patterns (%s) that have the same priority [%d], multiple index templates may not "
-                    + "match during index creation, please use a different priority",
-                name,
-                template.indexPatterns(),
-                Strings.collectionToCommaDelimitedString(overlaps.keySet()),
-                overlaps.entrySet().stream().map(e -> e.getKey() + " => " + e.getValue()).collect(Collectors.joining(",")),
-                template.priorityOrZero()
-            );
-            throw new IllegalArgumentException(error);
-        }
+        Map<String, List<String>> overlaps = v2TemplateOverlaps(currentState, name, template, validateV2Overlaps);
 
         overlaps = findConflictingV1Templates(currentState, name, template.indexPatterns());
         if (overlaps.size() > 0) {
@@ -615,6 +605,49 @@ public class MetadataIndexTemplateService {
         return ClusterState.builder(currentState).metadata(Metadata.builder(currentState.metadata()).put(name, finalIndexTemplate)).build();
     }
 
+    /**
+     * Calculates the conflicting v2 index template overlaps for a given composable index template. Optionally if validate is true
+     * we throw an {@link IllegalArgumentException} with information about the conflicting templates.
+     * <p>
+     * This method doesn't check for conflicting overlaps with v1 templates.
+     * @param currentState the current cluster state
+     * @param name the composable index template name
+     * @param template the full composable index template object we check for overlaps
+     * @param validate should we throw {@link IllegalArgumentException} if conflicts are found or just compute them
+     * @return a map of v2 template names to their index patterns for v2 templates that would overlap with the given template
+     */
+    public Map<String, List<String>> v2TemplateOverlaps(
+        ClusterState currentState,
+        String name,
+        final ComposableIndexTemplate template,
+        boolean validate
+    ) {
+        Map<String, List<String>> overlaps = findConflictingV2Templates(
+            currentState,
+            name,
+            template.indexPatterns(),
+            true,
+            template.priorityOrZero()
+        );
+        overlaps.remove(name);
+        if (validate && overlaps.size() > 0) {
+            String error = String.format(
+                Locale.ROOT,
+                "index template [%s] has index patterns %s matching patterns from "
+                    + "existing templates [%s] with patterns (%s) that have the same priority [%d], multiple index templates may not "
+                    + "match during index creation, please use a different priority",
+                name,
+                template.indexPatterns(),
+                Strings.collectionToCommaDelimitedString(overlaps.keySet()),
+                overlaps.entrySet().stream().map(e -> e.getKey() + " => " + e.getValue()).collect(Collectors.joining(",")),
+                template.priorityOrZero()
+            );
+            throw new IllegalArgumentException(error);
+        }
+
+        return overlaps;
+    }
+
     private void validateIndexTemplateV2(String name, ComposableIndexTemplate indexTemplate, ClusterState currentState) {
         // Workaround for the fact that start_time and end_time are injected by the MetadataCreateDataStreamService upon creation,
         // but when validating templates that create data streams the MetadataCreateDataStreamService isn't used.
@@ -887,8 +920,8 @@ public class MetadataIndexTemplateService {
         );
     }
 
-    // Package visible for testing
-    static ClusterState innerRemoveIndexTemplateV2(ClusterState currentState, String... names) {
+    // Public because it's used by ReservedComposableIndexTemplateAction
+    public static ClusterState innerRemoveIndexTemplateV2(ClusterState currentState, String... names) {
         Set<String> templateNames = new HashSet<>();
 
         if (names.length > 1) {

+ 14 - 0
server/src/main/java/org/elasticsearch/node/Node.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequest;
 import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.action.ActionType;
 import org.elasticsearch.action.admin.cluster.repositories.reservedstate.ReservedRepositoryAction;
+import org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction;
 import org.elasticsearch.action.ingest.ReservedPipelineAction;
 import org.elasticsearch.action.search.SearchExecutionStatsCollector;
 import org.elasticsearch.action.search.SearchPhaseController;
@@ -52,6 +53,7 @@ import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.metadata.MetadataCreateDataStreamService;
 import org.elasticsearch.cluster.metadata.MetadataCreateIndexService;
 import org.elasticsearch.cluster.metadata.MetadataDataStreamsService;
+import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
 import org.elasticsearch.cluster.metadata.MetadataUpdateSettingsService;
 import org.elasticsearch.cluster.metadata.SystemIndexMetadataUpgradeService;
 import org.elasticsearch.cluster.metadata.TemplateUpgradeService;
@@ -724,6 +726,18 @@ public class Node implements Closeable {
             // add all reserved state handlers from server
             reservedStateHandlers.add(new ReservedClusterSettingsAction(settingsModule.getClusterSettings()));
 
+            var templateService = new MetadataIndexTemplateService(
+                clusterService,
+                metadataCreateIndexService,
+                indicesService,
+                settingsModule.getIndexScopedSettings(),
+                xContentRegistry,
+                systemIndices,
+                indexSettingProviders
+            );
+
+            reservedStateHandlers.add(new ReservedComposableIndexTemplateAction(templateService, settingsModule.getIndexScopedSettings()));
+
             // add all reserved state handlers from plugins
             List<? extends ReservedClusterStateHandlerProvider> pluginHandlers = pluginsService.loadServiceProviders(
                 ReservedClusterStateHandlerProvider.class

+ 963 - 0
server/src/test/java/org/elasticsearch/action/admin/indices/template/reservedstate/ReservedComposableIndexTemplateActionTests.java

@@ -0,0 +1,963 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.action.admin.indices.template.reservedstate;
+
+import org.elasticsearch.action.admin.indices.template.delete.DeleteComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.delete.DeleteComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.PutComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.TransportPutComponentTemplateAction;
+import org.elasticsearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction;
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.MetadataCreateIndexService;
+import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
+import org.elasticsearch.cluster.metadata.ReservedStateHandlerMetadata;
+import org.elasticsearch.cluster.metadata.ReservedStateMetadata;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.IndexScopedSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.IndexSettingProviders;
+import org.elasticsearch.index.mapper.MapperService;
+import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.indices.InvalidIndexTemplateException;
+import org.elasticsearch.indices.SystemIndices;
+import org.elasticsearch.reservedstate.ActionWithReservedState;
+import org.elasticsearch.reservedstate.ReservedClusterStateHandler;
+import org.elasticsearch.reservedstate.TransformState;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xcontent.XContentParseException;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xcontent.XContentType;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.singletonList;
+import static org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction.reservedComponentName;
+import static org.elasticsearch.action.admin.indices.template.reservedstate.ReservedComposableIndexTemplateAction.reservedComposableIndexName;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasKey;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+/**
+ * A unit test class that tests {@link ReservedComposableIndexTemplateAction}
+ */
+public class ReservedComposableIndexTemplateActionTests extends ESTestCase {
+
+    MetadataIndexTemplateService templateService;
+    ClusterService clusterService;
+    IndexScopedSettings indexScopedSettings;
+    IndicesService indicesService;
+
+    @Before
+    public void setup() throws IOException {
+        clusterService = mock(ClusterService.class);
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).build();
+        doReturn(state).when(clusterService).state();
+
+        final Settings settings = Settings.builder().put(IndexMetadata.SETTING_CREATION_DATE, System.currentTimeMillis()).build();
+        indexScopedSettings = new IndexScopedSettings(settings, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS);
+
+        indicesService = mock(IndicesService.class);
+        var indexService = mock(IndexService.class);
+        var mapperService = mock(MapperService.class);
+        doReturn(mapperService).when(indexService).mapperService();
+        doReturn(indexService).when(indicesService).createIndex(any(), any(), anyBoolean());
+
+        templateService = new MetadataIndexTemplateService(
+            clusterService,
+            mock(MetadataCreateIndexService.class),
+            indicesService,
+            indexScopedSettings,
+            mock(NamedXContentRegistry.class),
+            mock(SystemIndices.class),
+            new IndexSettingProviders(Set.of())
+        );
+    }
+
+    private TransformState processJSON(
+        ReservedClusterStateHandler<ReservedComposableIndexTemplateAction.ComponentsAndComposables> action,
+        TransformState prevState,
+        String json
+    ) throws Exception {
+        try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, json)) {
+            return action.transform(action.fromXContent(parser), prevState);
+        }
+    }
+
+    public void testComponentValidation() {
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).build();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String badComponentJSON = """
+            {
+              "component_templates": {
+                "template_1": {
+                  "template": {
+                    "_mappings": {
+                      "properties": {
+                        "@timestamp": {
+                          "type": "date"
+                        }
+                      }
+                    }
+                  }
+                }
+              }
+            }""";
+
+        assertEquals(
+            "[1:26] [component_template] failed to parse field [template]",
+            expectThrows(XContentParseException.class, () -> processJSON(action, prevState, badComponentJSON)).getMessage()
+        );
+    }
+
+    public void testComposableIndexValidation() {
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).build();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String badComponentJSON = """
+            {
+              "composable_index_templates": {
+                "template_1": {
+                  "index_patterns": ["te*", "bar*"],
+                  "template": {
+                    "settings": {
+                      "number_of_shards": 1
+                    },
+                    "mappings": {
+                      "_source": {
+                        "enabled": true
+                      },
+                      "properties": {
+                        "host_name": {
+                          "type": "keyword"
+                        },
+                        "created_at": {
+                          "type": "date",
+                          "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                        }
+                      }
+                    },
+                    "aliases": {
+                      "mydata": { }
+                    }
+                  },
+                  "priority": -500,
+                  "composed_of": ["component_template1", "runtime_component_template"],
+                  "version": 3,
+                  "_meta": {
+                    "description": "my custom"
+                  }
+                }
+              }
+            }""";
+
+        assertEquals(
+            "Validation Failed: 1: index template priority must be >= 0;",
+            expectThrows(IllegalStateException.class, () -> processJSON(action, prevState, badComponentJSON)).getCause().getMessage()
+        );
+
+        String badComponentJSON1 = """
+            {
+              "composable_index_templates": {
+                "template_1": {
+                  "index_patterns": ["te*", "bar*"],
+                  "template": {
+                    "settings": {
+                      "number_of_shards": 1
+                    },
+                    "mappings": {
+                      "_source": {
+                        "enabled": true
+                      },
+                      "properties": {
+                        "host_name": {
+                          "type": "keyword"
+                        },
+                        "created_at": {
+                          "type": "date",
+                          "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                        }
+                      }
+                    },
+                    "aliases": {
+                      "mydata": { }
+                    }
+                  },
+                  "priority": 500,
+                  "composed_of": ["component_template1", "runtime_component_template"],
+                  "version": 3,
+                  "_meta": {
+                    "description": "my custom"
+                  }
+                }
+              }
+            }""";
+
+        assertEquals(
+            "index_template [template_1] invalid, cause [index template [template_1] specifies "
+                + "component templates [component_template1, runtime_component_template] that do not exist]",
+            expectThrows(InvalidIndexTemplateException.class, () -> processJSON(action, prevState, badComponentJSON1)).getMessage()
+        );
+    }
+
+    public void testAddRemoveComponentTemplates() throws Exception {
+        ClusterState state = clusterService.state();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String emptyJSON = "";
+
+        TransformState updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertEquals(prevState.state(), updatedState.state());
+
+        String settingsJSON = """
+            {
+              "component_templates": {
+                "template_1": {
+                  "template": {
+                    "mappings": {
+                      "properties": {
+                        "@timestamp": {
+                          "type": "date"
+                        }
+                      }
+                    }
+                  }
+                },
+                "template_2": {
+                  "template": {
+                    "mappings": {
+                      "runtime": {
+                        "day_of_week": {
+                          "type": "keyword",
+                          "script": {
+                            "source": "emit(doc['@timestamp'].value.dayOfWeekEnum.getDisplayName(TextStyle.FULL, Locale.ROOT))"
+                          }
+                        }
+                      }
+                    }
+                  }
+                }
+              }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, settingsJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder(reservedComponentName("template_1"), reservedComponentName("template_2")));
+
+        String lessJSON = """
+            {
+              "component_templates": {
+                "template_2": {
+                  "template": {
+                    "mappings": {
+                      "runtime": {
+                        "day_of_week": {
+                          "type": "keyword",
+                          "script": {
+                            "source": "emit(doc['@timestamp'].value.dayOfWeekEnum.getDisplayName(TextStyle.FULL, Locale.ROOT))"
+                          }
+                        }
+                      }
+                    }
+                  }
+                }
+              }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, lessJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder(reservedComponentName("template_2")));
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+    }
+
+    public void testAddRemoveIndexTemplates() throws Exception {
+        ClusterState state = clusterService.state();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String emptyJSON = "";
+
+        TransformState updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertEquals(prevState.state(), updatedState.state());
+
+        String settingsJSON = """
+            {
+              "component_templates": {
+                "component_template1": {
+                  "template": {
+                    "mappings": {
+                      "properties": {
+                        "@timestamp": {
+                          "type": "date"
+                        }
+                      }
+                    }
+                  }
+                },
+                "runtime_component_template": {
+                  "template": {
+                    "mappings": {
+                      "runtime": {
+                        "day_of_week": {
+                          "type": "keyword"
+                        }
+                      }
+                    }
+                  }
+                }
+              },
+              "composable_index_templates": {
+                "template_1": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "composed_of": ["component_template1", "runtime_component_template"],
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                },
+                "template_2": {
+                    "index_patterns": ["foo*", "mar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 100,
+                    "composed_of": ["component_template1", "runtime_component_template"],
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                }
+              }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, settingsJSON);
+        assertThat(
+            updatedState.keys(),
+            containsInAnyOrder(
+                reservedComposableIndexName("template_1"),
+                reservedComposableIndexName("template_2"),
+                reservedComponentName("component_template1"),
+                reservedComponentName("runtime_component_template")
+            )
+        );
+
+        String lessJSON = """
+            {
+              "component_templates": {
+                "component_template1": {
+                  "template": {
+                    "mappings": {
+                      "properties": {
+                        "@timestamp": {
+                          "type": "date"
+                        }
+                      }
+                    }
+                  }
+                },
+                "runtime_component_template": {
+                  "template": {
+                    "mappings": {
+                      "runtime": {
+                        "day_of_week": {
+                          "type": "keyword"
+                        }
+                      }
+                    }
+                  }
+                }
+              },
+              "composable_index_templates": {
+                "template_2": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "composed_of": ["component_template1", "runtime_component_template"],
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                }
+              }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, lessJSON);
+        assertThat(
+            updatedState.keys(),
+            containsInAnyOrder(
+                reservedComposableIndexName("template_2"),
+                reservedComponentName("component_template1"),
+                reservedComponentName("runtime_component_template")
+            )
+        );
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+    }
+
+    public void testAddRemoveIndexTemplatesWithOverlap() throws Exception {
+        ClusterState state = clusterService.state();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String emptyJSON = "";
+
+        TransformState updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertEquals(prevState.state(), updatedState.state());
+
+        // Adding two composable index templates with same index patterns will fail
+        String settingsJSON = """
+            {
+              "composable_index_templates": {
+                "template_1": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                },
+                "template_2": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                }
+              }
+            }""";
+
+        var prevState1 = updatedState;
+
+        assertTrue(
+            expectThrows(IllegalArgumentException.class, () -> processJSON(action, prevState1, settingsJSON)).getMessage()
+                .contains(
+                    "index template [template_2] has index patterns [te*, bar*] " + "matching patterns from existing templates [template_1]"
+                )
+        );
+
+        var newSettingsJSON = """
+            {
+              "composable_index_templates": {
+                "template_1": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                }
+              }
+            }""";
+
+        // We add one only to see if we can replace it subsequently, inserts happen before deletes in ReservedComposableIndexTemplateAction
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, newSettingsJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder(reservedComposableIndexName("template_1")));
+
+        String lessJSON = """
+            {
+              "composable_index_templates": {
+                "template_2": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                }
+              }
+            }""";
+
+        // We are replacing template_1 with template_2, same index pattern, no validation should be thrown
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, lessJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder(reservedComposableIndexName("template_2")));
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+    }
+
+    public void testHandlerCorrectness() {
+        var putIndexAction = new TransportPutComposableIndexTemplateAction(
+            mock(TransportService.class),
+            null,
+            null,
+            null,
+            mock(ActionFilters.class),
+            null
+        );
+        assertEquals(ReservedComposableIndexTemplateAction.NAME, putIndexAction.reservedStateHandlerName().get());
+        assertThat(
+            putIndexAction.modifiedKeys(new PutComposableIndexTemplateAction.Request("aaa")),
+            containsInAnyOrder(reservedComposableIndexName("aaa"))
+        );
+        var delIndexAction = new TransportDeleteComposableIndexTemplateAction(
+            mock(TransportService.class),
+            null,
+            null,
+            null,
+            mock(ActionFilters.class),
+            null
+        );
+        assertEquals(ReservedComposableIndexTemplateAction.NAME, delIndexAction.reservedStateHandlerName().get());
+        assertThat(
+            delIndexAction.modifiedKeys(new DeleteComposableIndexTemplateAction.Request("a", "b")),
+            containsInAnyOrder(reservedComposableIndexName("a"), reservedComposableIndexName("b"))
+        );
+
+        var putComponentAction = new TransportPutComponentTemplateAction(
+            mock(TransportService.class),
+            null,
+            null,
+            null,
+            mock(ActionFilters.class),
+            null,
+            indexScopedSettings
+        );
+        assertEquals(ReservedComposableIndexTemplateAction.NAME, putComponentAction.reservedStateHandlerName().get());
+        assertThat(
+            putComponentAction.modifiedKeys(new PutComponentTemplateAction.Request("aaa")),
+            containsInAnyOrder(reservedComponentName("aaa"))
+        );
+
+        var delComponentAction = new TransportDeleteComponentTemplateAction(
+            mock(TransportService.class),
+            null,
+            null,
+            null,
+            mock(ActionFilters.class),
+            null
+        );
+        assertEquals(ReservedComposableIndexTemplateAction.NAME, delComponentAction.reservedStateHandlerName().get());
+        assertThat(
+            delComponentAction.modifiedKeys(new DeleteComponentTemplateAction.Request("a", "b")),
+            containsInAnyOrder(reservedComponentName("a"), reservedComponentName("b"))
+        );
+    }
+
+    public void testBlockUsingReservedComponentTemplates() throws Exception {
+        ClusterState state = clusterService.state();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(templateService, indexScopedSettings);
+
+        String settingsJSON = """
+            {
+              "component_templates": {
+                "template_1": {
+                  "template": {
+                    "mappings": {
+                      "properties": {
+                        "@timestamp": {
+                          "type": "date"
+                        }
+                      }
+                    }
+                  }
+                }
+              }
+            }""";
+
+        var updatedState = processJSON(action, prevState, settingsJSON);
+
+        Metadata metadata = Metadata.builder(updatedState.state().metadata())
+            .put(
+                ReservedStateMetadata.builder("test")
+                    .putHandler(new ReservedStateHandlerMetadata(ReservedComposableIndexTemplateAction.NAME, updatedState.keys()))
+                    .build()
+            )
+            .build();
+
+        ClusterState withReservedState = new ClusterState.Builder(updatedState.state()).metadata(metadata).build();
+
+        String composableTemplate = """
+            {
+              "composable_index_templates": {
+                "composable_template_1": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "composed_of": ["%s"],
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                  }
+                }
+              }
+            }""";
+
+        try (
+            XContentParser parser = XContentType.JSON.xContent()
+                .createParser(XContentParserConfiguration.EMPTY, String.format(Locale.ROOT, composableTemplate, "template_1"))
+        ) {
+            var request = action.fromXContent(parser).composableTemplates().get(0);
+            assertTrue(
+                expectThrows(
+                    IllegalArgumentException.class,
+                    () -> TransportPutComposableIndexTemplateAction.verifyIfUsingReservedComponentTemplates(request, withReservedState)
+                ).getMessage().contains("errors: [[component_template:template_1] is reserved by [test]]")
+            );
+        }
+
+        try (
+            XContentParser parser = XContentType.JSON.xContent()
+                .createParser(XContentParserConfiguration.EMPTY, String.format(Locale.ROOT, composableTemplate, "template_2"))
+        ) {
+            var request = action.fromXContent(parser).composableTemplates().get(0);
+            // this should just work, no failure
+            TransportPutComposableIndexTemplateAction.verifyIfUsingReservedComponentTemplates(request, withReservedState);
+        }
+    }
+
+    public void testTemplatesWithReservedPrefix() throws Exception {
+        final String conflictingTemplateName = "validate_template";
+
+        // Reserve the validate_template name in the reserved metadata
+        String composableTemplate = String.format(Locale.ROOT, """
+            {
+              "composable_index_templates": {
+                "%s": {
+                    "index_patterns": ["te*", "bar*"],
+                    "template": {
+                      "settings": {
+                        "number_of_shards": 1
+                      },
+                      "mappings": {
+                        "_source": {
+                          "enabled": true
+                        },
+                        "properties": {
+                          "host_name": {
+                            "type": "keyword"
+                          },
+                          "created_at": {
+                            "type": "date",
+                            "format": "EEE MMM dd HH:mm:ss Z yyyy"
+                          }
+                        }
+                      },
+                      "aliases": {
+                        "mydata": { }
+                      }
+                    },
+                    "priority": 500,
+                    "version": 3,
+                    "_meta": {
+                      "description": "my custom"
+                    }
+                  }
+                }
+              }
+            }""", conflictingTemplateName);
+
+        // add a non-reserved template into the cluster state that has a name of validate_template, but with the composable
+        // index name prefix.
+        Metadata metadata = Metadata.builder()
+            .indexTemplates(
+                Map.of(
+                    reservedComposableIndexName(conflictingTemplateName),
+                    new ComposableIndexTemplate(singletonList("foo*"), null, Collections.emptyList(), 1L, 1L, Collections.emptyMap())
+                )
+            )
+            .build();
+
+        ClusterService mockedClusterService = spy(clusterService);
+        MetadataIndexTemplateService mockedTemplateService = new MetadataIndexTemplateService(
+            mockedClusterService,
+            mock(MetadataCreateIndexService.class),
+            indicesService,
+            indexScopedSettings,
+            mock(NamedXContentRegistry.class),
+            mock(SystemIndices.class),
+            new IndexSettingProviders(Set.of())
+        );
+
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).metadata(metadata).build();
+        doReturn(state).when(mockedClusterService).state();
+
+        // we should see the weird composable name prefixed 'validate_template'
+        assertThat(state.metadata().templatesV2(), allOf(aMapWithSize(1), hasKey(reservedComposableIndexName(conflictingTemplateName))));
+
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        var action = new ReservedComposableIndexTemplateAction(mockedTemplateService, indexScopedSettings);
+
+        TransformState updatedState = processJSON(action, prevState, composableTemplate);
+
+        // only one reserved key for 'validate_template'
+        assertThat(updatedState.keys(), containsInAnyOrder(reservedComposableIndexName(conflictingTemplateName)));
+        // we should find a template name with 'validate_template' and 'composable_index_template:validate_template'. The user had
+        // added that weird name 'composable_index_template:validate_template', using this prefix in the name shouldn't make us fail
+        // any reservation validation
+        assertThat(
+            updatedState.state().metadata().templatesV2(),
+            allOf(aMapWithSize(2), hasKey(reservedComposableIndexName(conflictingTemplateName)), hasKey(conflictingTemplateName))
+        );
+
+        Metadata withReservedMetadata = Metadata.builder(updatedState.state().metadata())
+            .put(
+                new ReservedStateMetadata.Builder("file_settings").putHandler(
+                    new ReservedStateHandlerMetadata(ReservedComposableIndexTemplateAction.NAME, updatedState.keys())
+                ).build()
+            )
+            .build();
+
+        // apply the modified keys to a cluster state, as the ReservedStateService would do
+        ClusterState withReservedState = new ClusterState.Builder(updatedState.state()).metadata(withReservedMetadata).build();
+
+        PutComposableIndexTemplateAction.Request pr = new PutComposableIndexTemplateAction.Request(conflictingTemplateName);
+
+        var putTemplateAction = new TransportPutComposableIndexTemplateAction(
+            mock(TransportService.class),
+            null,
+            null,
+            null,
+            mock(ActionFilters.class),
+            null
+        );
+
+        // Try fake REST modification request with validate_template, this will fail
+        var modifiedKeys = putTemplateAction.modifiedKeys(pr);
+        assertEquals(1, modifiedKeys.size());
+
+        var fakeAction = new ActionWithReservedState<PutComposableIndexTemplateAction.Request>() {
+        };
+        assertEquals(
+            "Failed to process request [validate_template] with errors: "
+                + "[[composable_index_template:validate_template] set as read-only by [file_settings]]",
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> fakeAction.validateForReservedState(
+                    withReservedState,
+                    ReservedComposableIndexTemplateAction.NAME,
+                    modifiedKeys,
+                    pr.name()
+                )
+            ).getMessage()
+        );
+
+        // Try fake REST modification request with the weird prefixed composable_index_template:validate_template, this will work, since
+        // the reserved keys for that name would be composable_index_template:composable_index_template:validate_template and it will not
+        // match our reserved state.
+        var prOK = new PutComposableIndexTemplateAction.Request(reservedComposableIndexName(conflictingTemplateName));
+        var modifiedKeysOK = putTemplateAction.modifiedKeys(prOK);
+        assertEquals(1, modifiedKeysOK.size());
+
+        fakeAction.validateForReservedState(withReservedState, ReservedComposableIndexTemplateAction.NAME, modifiedKeysOK, prOK.name());
+    }
+}

+ 100 - 0
server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java

@@ -55,12 +55,15 @@ import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.DE
 import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.innerRemoveComponentTemplate;
 import static org.elasticsearch.common.settings.Settings.builder;
 import static org.elasticsearch.indices.ShardLimitValidatorTests.createTestShardLimitService;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.containsStringIgnoringCase;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasKey;
 import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.matchesRegex;
@@ -2007,6 +2010,103 @@ public class MetadataIndexTemplateServiceTests extends ESSingleNodeTestCase {
         MetadataIndexTemplateService.innerRemoveIndexTemplateV2(stateWithTwoTemplates, "logs");
     }
 
+    public void testV2TemplateOverlaps() throws Exception {
+        {
+            ComposableIndexTemplate template = new ComposableIndexTemplate(
+                Arrays.asList("egg*", "baz"),
+                null,
+                null,
+                1L,
+                null,
+                null,
+                null,
+                null
+            );
+            MetadataIndexTemplateService metadataIndexTemplateService = getMetadataIndexTemplateService();
+            ClusterState state = metadataIndexTemplateService.addIndexTemplateV2(ClusterState.EMPTY_STATE, false, "foo", template);
+            ComposableIndexTemplate newTemplate = new ComposableIndexTemplate(
+                Arrays.asList("abc", "baz*"),
+                null,
+                null,
+                1L,
+                null,
+                null,
+                null,
+                null
+            );
+
+            // when validating is false, we return the conflicts instead of throwing an exception
+            var overlaps = metadataIndexTemplateService.v2TemplateOverlaps(state, "foo2", newTemplate, false);
+
+            assertThat(overlaps, allOf(aMapWithSize(1), hasKey("foo")));
+
+            // try now the same thing with validation on
+            IllegalArgumentException e = expectThrows(
+                IllegalArgumentException.class,
+                () -> metadataIndexTemplateService.v2TemplateOverlaps(state, "foo2", newTemplate, true)
+            );
+            assertThat(
+                e.getMessage(),
+                equalTo(
+                    "index template [foo2] has index patterns [abc, baz*] matching patterns from existing "
+                        + "templates [foo] with patterns (foo => [egg*, baz]) that have the same priority [1], multiple "
+                        + "index templates may not match during index creation, please use a different priority"
+                )
+            );
+
+            ComposableIndexTemplate nonConflict = new ComposableIndexTemplate(
+                Arrays.asList("abc", "bar*"),
+                null,
+                null,
+                1L,
+                null,
+                null,
+                null,
+                null
+            );
+
+            overlaps = metadataIndexTemplateService.v2TemplateOverlaps(state, "no-conflict", nonConflict, true);
+            assertTrue(overlaps.isEmpty());
+        }
+
+        {
+            ComposableIndexTemplate template = new ComposableIndexTemplate(
+                Arrays.asList("egg*", "baz"),
+                null,
+                null,
+                null,
+                null,
+                null,
+                null,
+                null
+            );
+            MetadataIndexTemplateService metadataIndexTemplateService = getMetadataIndexTemplateService();
+            ClusterState state = metadataIndexTemplateService.addIndexTemplateV2(ClusterState.EMPTY_STATE, false, "foo", template);
+            ComposableIndexTemplate newTemplate = new ComposableIndexTemplate(
+                Arrays.asList("abc", "baz*"),
+                null,
+                null,
+                0L,
+                null,
+                null,
+                null,
+                null
+            );
+            IllegalArgumentException e = expectThrows(
+                IllegalArgumentException.class,
+                () -> metadataIndexTemplateService.v2TemplateOverlaps(state, "foo2", newTemplate, true)
+            );
+            assertThat(
+                e.getMessage(),
+                equalTo(
+                    "index template [foo2] has index patterns [abc, baz*] matching patterns from existing "
+                        + "templates [foo] with patterns (foo => [egg*, baz]) that have the same priority [0], multiple "
+                        + "index templates may not match during index creation, please use a different priority"
+                )
+            );
+        }
+    }
+
     private static List<Throwable> putTemplate(NamedXContentRegistry xContentRegistry, PutRequest request) {
         ThreadPool testThreadPool = mock(ThreadPool.class);
         ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool);