Explorar o código

Deprecate creation of dot-prefixed index names except for hidden and system indices (#49959)

This commit deprecates the creation of dot-prefixed index names (e.g.
.watches) unless they are either 1) a hidden index, or 2) registered by
a plugin that extends SystemIndexPlugin. This is the first step
towards more thorough protections for system indices.

This commit also modifies several plugins which use dot-prefixed indices
to register indices they own as system indices, and adds a plugin to
register .tasks as a system index.
Gordon Brown %!s(int64=5) %!d(string=hai) anos
pai
achega
44f5ed6fd9
Modificáronse 27 ficheiros con 663 adicións e 54 borrados
  1. 1 1
      docs/reference/index-modules.asciidoc
  2. 2 1
      docs/reference/indices/create-index.asciidoc
  3. 25 0
      modules/tasks/build.gradle
  4. 40 0
      modules/tasks/src/main/java/org/elasticsearch/tasksplugin/TasksPlugin.java
  5. 32 0
      modules/tasks/src/test/java/org/elasticsearch/tasksplugin/TasksPluginTests.java
  6. 1 1
      server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java
  7. 43 12
      server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java
  8. 141 0
      server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java
  9. 16 1
      server/src/main/java/org/elasticsearch/node/Node.java
  10. 41 0
      server/src/main/java/org/elasticsearch/plugins/SystemIndexPlugin.java
  11. 2 1
      server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
  12. 2 0
      server/src/test/java/org/elasticsearch/action/admin/indices/template/put/MetaDataIndexTemplateServiceTests.java
  13. 85 12
      server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java
  14. 117 0
      server/src/test/java/org/elasticsearch/indices/SystemIndexDescriptorTests.java
  15. 1 1
      server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java
  16. 1 1
      server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
  17. 1 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/enrich/EnrichPolicy.java
  18. 1 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/annotations/AnnotationIndex.java
  19. 5 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java
  20. 12 2
      x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPlugin.java
  21. 12 3
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java
  22. 12 2
      x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/Logstash.java
  23. 21 5
      x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java
  24. 16 2
      x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java
  25. 5 5
      x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml
  26. 13 2
      x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java
  27. 15 2
      x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java

+ 1 - 1
docs/reference/index-modules.asciidoc

@@ -79,7 +79,7 @@ indices.
     Indicates whether <<query-filter-context, cached filters>> are pre-loaded for
     nested queries. Possible values are `true` (default) and `false`.
 
-`index.hidden`::
+[[index-hidden]] `index.hidden`::
 
     Indicates whether the index should be hidden by default. Hidden indices are not
     returned by default when using a wildcard expression. This behavior is controlled

+ 2 - 1
docs/reference/indices/create-index.asciidoc

@@ -19,7 +19,7 @@ PUT /twitter
 
 [[indices-create-api-desc]]
 ==== {api-description-title}
-You can use the create index API to add a new index to an {es} cluster. When 
+You can use the create index API to add a new index to an {es} cluster. When
 creating an index, you can specify the following:
 
 * Settings for the index
@@ -44,6 +44,7 @@ Index names must meet the following criteria:
 - Cannot start with `-`, `_`, `+`
 - Cannot be `.` or `..`
 - Cannot be longer than 255 bytes (note it is bytes, so multi-byte characters will count towards the 255 limit faster)
+- Names starting with `.` are deprecated, except for <<index-hidden,hidden indices>> and internal indices managed by plugins
 // end::index-name-reqs[]
 --
 

+ 25 - 0
modules/tasks/build.gradle

@@ -0,0 +1,25 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+esplugin {
+  description 'Supports the Tasks API'
+  classname 'org.elasticsearch.tasksplugin.TasksPlugin'
+}
+
+integTest.enabled = false

+ 40 - 0
modules/tasks/src/main/java/org/elasticsearch/tasksplugin/TasksPlugin.java

@@ -0,0 +1,40 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.tasksplugin;
+
+import org.elasticsearch.indices.SystemIndexDescriptor;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.elasticsearch.tasks.TaskResultsService.TASK_INDEX;
+
+/**
+ * This plugin currently only exists to register `.tasks` as a system index.
+ */
+public class TasksPlugin extends Plugin implements SystemIndexPlugin {
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.singletonList(new SystemIndexDescriptor(TASK_INDEX, this.getClass().getSimpleName()));
+    }
+}

+ 32 - 0
modules/tasks/src/test/java/org/elasticsearch/tasksplugin/TasksPluginTests.java

@@ -0,0 +1,32 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.tasksplugin;
+
+import org.elasticsearch.test.ESTestCase;
+import org.hamcrest.Matchers;
+
+public class TasksPluginTests extends ESTestCase {
+
+    public void testDummy() {
+        // This is a dummy test case to satisfy the conventions
+        TasksPlugin plugin = new TasksPlugin();
+        assertThat(plugin.getSystemIndexDescriptors(), Matchers.hasSize(1));
+    }
+}

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java

@@ -122,9 +122,9 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
             ? rolloverRequest.getNewIndexName()
             : generateRolloverIndexName(sourceProvidedName, indexNameExpressionResolver);
         final String rolloverIndexName = indexNameExpressionResolver.resolveDateMathExpression(unresolvedName);
-        MetaDataCreateIndexService.validateIndexName(rolloverIndexName, state); // will fail if the index already exists
         final Boolean isHidden = IndexMetaData.INDEX_HIDDEN_SETTING.exists(rolloverRequest.getCreateIndexRequest().settings()) ?
             IndexMetaData.INDEX_HIDDEN_SETTING.get(rolloverRequest.getCreateIndexRequest().settings()) : null;
+        createIndexService.validateIndexName(rolloverIndexName, state, isHidden); // fails if the index already exists
         checkNoDuplicatedAliasInIndexTemplate(metaData, rolloverIndexName, rolloverRequest.getAlias(), isHidden);
         IndicesStatsRequest statsRequest = new IndicesStatsRequest().indices(rolloverRequest.getAlias())
             .clear()

+ 43 - 12
server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java

@@ -53,6 +53,7 @@ import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.ValidationException;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.io.PathUtils;
+import org.elasticsearch.common.logging.DeprecationLogger;
 import org.elasticsearch.common.settings.IndexScopedSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
@@ -70,6 +71,7 @@ import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.indices.IndexCreationException;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.InvalidIndexNameException;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason;
 import org.elasticsearch.threadpool.ThreadPool;
 
@@ -78,6 +80,7 @@ import java.io.UnsupportedEncodingException;
 import java.nio.file.Path;
 import java.time.Instant;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -89,6 +92,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BiFunction;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 import static java.util.stream.Collectors.toList;
@@ -103,6 +107,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF
  */
 public class MetaDataCreateIndexService {
     private static final Logger logger = LogManager.getLogger(MetaDataCreateIndexService.class);
+    private static final DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
 
     public static final int MAX_INDEX_NAME_BYTES = 255;
 
@@ -115,19 +120,21 @@ public class MetaDataCreateIndexService {
     private final IndexScopedSettings indexScopedSettings;
     private final ActiveShardsObserver activeShardsObserver;
     private final NamedXContentRegistry xContentRegistry;
+    private final Collection<SystemIndexDescriptor> systemIndexDescriptors;
     private final boolean forbidPrivateIndexSettings;
 
     public MetaDataCreateIndexService(
-            final Settings settings,
-            final ClusterService clusterService,
-            final IndicesService indicesService,
-            final AllocationService allocationService,
-            final AliasValidator aliasValidator,
-            final Environment env,
-            final IndexScopedSettings indexScopedSettings,
-            final ThreadPool threadPool,
-            final NamedXContentRegistry xContentRegistry,
-            final boolean forbidPrivateIndexSettings) {
+        final Settings settings,
+        final ClusterService clusterService,
+        final IndicesService indicesService,
+        final AllocationService allocationService,
+        final AliasValidator aliasValidator,
+        final Environment env,
+        final IndexScopedSettings indexScopedSettings,
+        final ThreadPool threadPool,
+        final NamedXContentRegistry xContentRegistry,
+        final Collection<SystemIndexDescriptor> systemIndexDescriptors,
+        final boolean forbidPrivateIndexSettings) {
         this.settings = settings;
         this.clusterService = clusterService;
         this.indicesService = indicesService;
@@ -137,17 +144,40 @@ public class MetaDataCreateIndexService {
         this.indexScopedSettings = indexScopedSettings;
         this.activeShardsObserver = new ActiveShardsObserver(clusterService, threadPool);
         this.xContentRegistry = xContentRegistry;
+        this.systemIndexDescriptors = systemIndexDescriptors;
         this.forbidPrivateIndexSettings = forbidPrivateIndexSettings;
     }
 
     /**
      * Validate the name for an index against some static rules and a cluster state.
      */
-    public static void validateIndexName(String index, ClusterState state) {
+    public void validateIndexName(String index, ClusterState state, @Nullable Boolean isHidden) {
         validateIndexOrAliasName(index, InvalidIndexNameException::new);
         if (!index.toLowerCase(Locale.ROOT).equals(index)) {
             throw new InvalidIndexNameException(index, "must be lowercase");
         }
+
+        if (index.charAt(0) == '.') {
+            List<SystemIndexDescriptor> matchingDescriptors = systemIndexDescriptors.stream()
+                .filter(descriptor -> descriptor.matchesIndexPattern(index))
+                .collect(toList());
+            if (matchingDescriptors.isEmpty() && (isHidden == null || isHidden == Boolean.FALSE)) {
+                deprecationLogger.deprecated("index name [{}] starts with a dot '.', in the next major version, index names " +
+                    "starting with a dot are reserved for hidden indices and system indices", index);
+            } else if (matchingDescriptors.size() > 1) {
+                // This should be prevented by erroring on overlapping patterns at startup time, but is here just in case.
+                StringBuilder errorMessage = new StringBuilder()
+                    .append("index name [")
+                    .append(index)
+                    .append("] is claimed as a system index by multiple system index patterns: [")
+                    .append(matchingDescriptors.stream()
+                        .map(descriptor -> "pattern: [" + descriptor.getIndexPattern() +
+                            "], description: [" + descriptor.getDescription() + "]").collect(Collectors.joining("; ")));
+                // Throw AssertionError if assertions are enabled, or a regular exception otherwise:
+                assert false : errorMessage.toString();
+                throw new IllegalStateException(errorMessage.toString());
+            }
+        }
         if (state.routingTable().hasIndex(index)) {
             throw new ResourceAlreadyExistsException(state.routingTable().index(index).getIndex());
         }
@@ -653,7 +683,8 @@ public class MetaDataCreateIndexService {
     }
 
     private void validate(CreateIndexClusterStateUpdateRequest request, ClusterState state) {
-        validateIndexName(request.index(), state);
+        boolean isHidden = IndexMetaData.INDEX_HIDDEN_SETTING.get(request.settings());
+        validateIndexName(request.index(), state, isHidden);
         validateIndexSettings(request.index(), request.settings(), forbidPrivateIndexSettings);
     }
 

+ 141 - 0
server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java

@@ -0,0 +1,141 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.indices;
+
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.regex.Regex;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Describes a system index. Provides the information required to create and maintain the system index.
+ */
+public class SystemIndexDescriptor {
+    private final String indexPattern;
+    private final String description;
+    private final CharacterRunAutomaton indexPatternAutomaton;
+
+    /**
+     *
+     * @param indexPattern The pattern of index names that this descriptor will be used for. Must start with a '.' character.
+     * @param description The name of the plugin responsible for this system index.
+     */
+    public SystemIndexDescriptor(String indexPattern, String description) {
+        Objects.requireNonNull(indexPattern, "system index pattern must not be null");
+        if (indexPattern.length() < 2) {
+            throw new IllegalArgumentException("system index pattern provided as [" + indexPattern +
+                "] but must at least 2 characters in length");
+        }
+        if (indexPattern.charAt(0) != '.') {
+            throw new IllegalArgumentException("system index pattern provided as [" + indexPattern +
+                "] but must start with the character [.]");
+        }
+        if (indexPattern.charAt(1) == '*') {
+            throw new IllegalArgumentException("system index pattern provided as [" + indexPattern +
+                "] but must not start with the character sequence [.*] to prevent conflicts");
+        }
+        this.indexPattern = indexPattern;
+        this.indexPatternAutomaton = new CharacterRunAutomaton(Regex.simpleMatchToAutomaton(indexPattern));
+        this.description = description;
+    }
+
+    /**
+     * @return The pattern of index names that this descriptor will be used for.
+     */
+    public String getIndexPattern() {
+        return indexPattern;
+    }
+
+    /**
+     * Checks whether an index name matches the system index name pattern for this descriptor.
+     * @param index The index name to be checked against the index pattern given at construction time.
+     * @return True if the name matches the pattern, false otherwise.
+     */
+    public boolean matchesIndexPattern(String index) {
+        return indexPatternAutomaton.run(index);
+    }
+
+    /**
+     * @return A short description of the purpose of this system index.
+     */
+    public String getDescription() {
+        return description;
+    }
+
+    @Override
+    public String toString() {
+        return "SystemIndexDescriptor[pattern=[" + indexPattern + "], description=[" + description + "]]";
+    }
+
+    /**
+     * Given a collection of {@link SystemIndexDescriptor}s and their sources, checks to see if the index patterns of the listed
+     * descriptors overlap with any of the other patterns. If any do, throws an exception.
+     *
+     * @param sourceToDescriptors A map of source (plugin) names to the SystemIndexDescriptors they provide.
+     * @throws IllegalStateException Thrown if any of the index patterns overlaps with another.
+     */
+    public static void checkForOverlappingPatterns(Map<String, Collection<SystemIndexDescriptor>> sourceToDescriptors) {
+        List<Tuple<String, SystemIndexDescriptor>> sourceDescriptorPair = sourceToDescriptors.entrySet().stream()
+            .flatMap(entry -> entry.getValue().stream().map(descriptor -> new Tuple<>(entry.getKey(), descriptor)))
+            .sorted(Comparator.comparing(d -> d.v1() + ":" + d.v2().getIndexPattern())) // Consistent ordering -> consistent error message
+            .collect(Collectors.toList());
+
+        // This is O(n^2) with the number of system index descriptors, and each check is quadratic with the number of states in the
+        // automaton, but the absolute number of system index descriptors should be quite small (~10s at most), and the number of states
+        // per pattern should be low as well. If these assumptions change, this might need to be reworked.
+        sourceDescriptorPair.forEach(descriptorToCheck -> {
+            List<Tuple<String, SystemIndexDescriptor>> descriptorsMatchingThisPattern = sourceDescriptorPair.stream()
+
+                .filter(d -> descriptorToCheck.v2() != d.v2()) // Exclude the pattern currently being checked
+                .filter(d -> overlaps(descriptorToCheck.v2(), d.v2()))
+                .collect(Collectors.toList());
+            if (descriptorsMatchingThisPattern.isEmpty() == false) {
+                StringBuilder errorMessage = new StringBuilder();
+                errorMessage.append("a system index descriptor [")
+                    .append(descriptorToCheck.v2())
+                    .append("] from plugin [")
+                    .append(descriptorToCheck.v1())
+                    .append("] overlaps with other system index descriptors: [")
+                    .append(descriptorsMatchingThisPattern.stream()
+                        .map(descriptor -> descriptor.v2() + " from plugin [" + descriptor.v1() + "]")
+                        .collect(Collectors.joining(", ")));
+                throw new IllegalStateException(errorMessage.toString());
+            }
+        });
+    }
+
+    private static boolean overlaps(SystemIndexDescriptor a1, SystemIndexDescriptor a2) {
+        Automaton a1Automaton = Regex.simpleMatchToAutomaton(a1.getIndexPattern());
+        Automaton a2Automaton = Regex.simpleMatchToAutomaton(a2.getIndexPattern());
+        return Operations.isEmpty(Operations.intersection(a1Automaton, a2Automaton)) == false;
+    }
+
+    // TODO: Index settings and mapping
+    // TODO: getThreadpool()
+    // TODO: Upgrade handling (reindex script?)
+}

+ 16 - 1
server/src/main/java/org/elasticsearch/node/Node.java

@@ -96,14 +96,15 @@ import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.gateway.GatewayMetaState;
 import org.elasticsearch.gateway.GatewayModule;
 import org.elasticsearch.gateway.GatewayService;
-import org.elasticsearch.gateway.PersistedClusterStateService;
 import org.elasticsearch.gateway.MetaStateService;
+import org.elasticsearch.gateway.PersistedClusterStateService;
 import org.elasticsearch.http.HttpServerTransport;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.analysis.AnalysisRegistry;
 import org.elasticsearch.index.engine.EngineFactory;
 import org.elasticsearch.indices.IndicesModule;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.analysis.AnalysisModule;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
 import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
@@ -136,6 +137,7 @@ import org.elasticsearch.plugins.PluginsService;
 import org.elasticsearch.plugins.RepositoryPlugin;
 import org.elasticsearch.plugins.ScriptPlugin;
 import org.elasticsearch.plugins.SearchPlugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.repositories.RepositoriesModule;
 import org.elasticsearch.repositories.RepositoriesService;
 import org.elasticsearch.rest.RestController;
@@ -429,6 +431,18 @@ public class Node implements Closeable {
                             .flatMap(m -> m.entrySet().stream())
                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
 
+            final Map<String, Collection<SystemIndexDescriptor>> systemIndexDescriptorMap = pluginsService
+                .filterPlugins(SystemIndexPlugin.class)
+                .stream()
+                .collect(Collectors.toUnmodifiableMap(
+                    plugin -> plugin.getClass().getSimpleName(),
+                    plugin -> plugin.getSystemIndexDescriptors()));
+            SystemIndexDescriptor.checkForOverlappingPatterns(systemIndexDescriptorMap);
+
+            final List<SystemIndexDescriptor> systemIndexDescriptors = systemIndexDescriptorMap.values().stream()
+                .flatMap(Collection::stream)
+                .collect(Collectors.toList());
+
             final IndicesService indicesService =
                 new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry, analysisModule.getAnalysisRegistry(),
                     clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
@@ -447,6 +461,7 @@ public class Node implements Closeable {
                     settingsModule.getIndexScopedSettings(),
                     threadPool,
                     xContentRegistry,
+                    systemIndexDescriptors,
                     forbidPrivateIndexSettings);
 
             Collection<Object> pluginComponents = pluginsService.filterPlugins(Plugin.class).stream()

+ 41 - 0
server/src/main/java/org/elasticsearch/plugins/SystemIndexPlugin.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.plugins;
+
+import org.elasticsearch.indices.SystemIndexDescriptor;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Plugin for defining system indices. Extends {@link ActionPlugin} because system indices must be accessed via APIs
+ * added by the plugin that owns the system index, rather than standard APIs.
+ */
+public interface SystemIndexPlugin extends ActionPlugin {
+
+    /**
+     * Returns a {@link Collection} of {@link SystemIndexDescriptor}s that describe this plugin's system indices, including
+     * name, mapping, and settings.
+     * @return Descriptions of the system indices managed by this plugin.
+     */
+    default Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.emptyList();
+    }
+}

+ 2 - 1
server/src/main/java/org/elasticsearch/snapshots/RestoreService.java

@@ -269,7 +269,8 @@ public class RestoreService implements ClusterStateApplier {
                                 if (currentIndexMetaData == null) {
                                     // Index doesn't exist - create it and start recovery
                                     // Make sure that the index we are about to create has a validate name
-                                    MetaDataCreateIndexService.validateIndexName(renamedIndexName, currentState);
+                                    boolean isHidden = IndexMetaData.INDEX_HIDDEN_SETTING.get(snapshotIndexMetaData.getSettings());
+                                    createIndexService.validateIndexName(renamedIndexName, currentState, isHidden);
                                     createIndexService.validateIndexSettings(renamedIndexName, snapshotIndexMetaData.getSettings(), false);
                                     IndexMetaData.Builder indexMdBuilder = IndexMetaData.builder(snapshotIndexMetaData)
                                         .state(IndexMetaData.State.OPEN)

+ 2 - 0
server/src/test/java/org/elasticsearch/action/admin/indices/template/put/MetaDataIndexTemplateServiceTests.java

@@ -213,6 +213,7 @@ public class MetaDataIndexTemplateServiceTests extends ESSingleNodeTestCase {
                 IndexScopedSettings.DEFAULT_SCOPED_SETTINGS,
                 null,
                 xContentRegistry,
+                Collections.emptyList(),
                 true);
         MetaDataIndexTemplateService service = new MetaDataIndexTemplateService(null, createIndexService,
                 new AliasValidator(), null,
@@ -246,6 +247,7 @@ public class MetaDataIndexTemplateServiceTests extends ESSingleNodeTestCase {
                 IndexScopedSettings.DEFAULT_SCOPED_SETTINGS,
                 null,
                 xContentRegistry(),
+                Collections.emptyList(),
                 true);
         MetaDataIndexTemplateService service = new MetaDataIndexTemplateService(
                 clusterService, createIndexService, new AliasValidator(), indicesService,

+ 85 - 12
server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java

@@ -56,9 +56,13 @@ import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.indices.InvalidAliasNameException;
 import org.elasticsearch.indices.InvalidIndexNameException;
+import org.elasticsearch.indices.SystemIndexDescriptor;
+import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.VersionUtils;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
+import org.elasticsearch.threadpool.TestThreadPool;
+import org.elasticsearch.threadpool.ThreadPool;
 import org.hamcrest.Matchers;
 import org.junit.Before;
 
@@ -457,26 +461,43 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase {
     }
 
     public void testValidateIndexName() throws Exception {
+        ThreadPool testThreadPool = new TestThreadPool(getTestName());
+        try {
+            MetaDataCreateIndexService checkerService = new MetaDataCreateIndexService(
+                Settings.EMPTY,
+                ClusterServiceUtils.createClusterService(testThreadPool),
+                null,
+                null,
+                null,
+                null,
+                null,
+                testThreadPool,
+                null,
+                Collections.emptyList(),
+                false
+            );
+            validateIndexName(checkerService, "index?name", "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS);
 
-        validateIndexName("index?name", "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS);
-
-        validateIndexName("index#name", "must not contain '#'");
+            validateIndexName(checkerService, "index#name", "must not contain '#'");
 
-        validateIndexName("_indexname", "must not start with '_', '-', or '+'");
-        validateIndexName("-indexname", "must not start with '_', '-', or '+'");
-        validateIndexName("+indexname", "must not start with '_', '-', or '+'");
+            validateIndexName(checkerService, "_indexname", "must not start with '_', '-', or '+'");
+            validateIndexName(checkerService, "-indexname", "must not start with '_', '-', or '+'");
+            validateIndexName(checkerService, "+indexname", "must not start with '_', '-', or '+'");
 
-        validateIndexName("INDEXNAME", "must be lowercase");
+            validateIndexName(checkerService, "INDEXNAME", "must be lowercase");
 
-        validateIndexName("..", "must not be '.' or '..'");
+            validateIndexName(checkerService, "..", "must not be '.' or '..'");
 
-        validateIndexName("foo:bar", "must not contain ':'");
+            validateIndexName(checkerService, "foo:bar", "must not contain ':'");
+        } finally {
+            testThreadPool.shutdown();
+        }
     }
 
-    private void validateIndexName(String indexName, String errorMessage) {
+    private void validateIndexName(MetaDataCreateIndexService metaDataCreateIndexService, String indexName, String errorMessage) {
         InvalidIndexNameException e = expectThrows(InvalidIndexNameException.class,
-            () -> MetaDataCreateIndexService.validateIndexName(indexName, ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING
-                .getDefault(Settings.EMPTY)).build()));
+            () -> metaDataCreateIndexService.validateIndexName(indexName, ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING
+                .getDefault(Settings.EMPTY)).build(), false));
         assertThat(e.getMessage(), endsWith(errorMessage));
     }
 
@@ -539,6 +560,58 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase {
         assertThat(e, hasToString(containsString(expectedMessage)));
     }
 
+    public void testValidateIndexNameChecksSystemIndexNames() {
+        List<SystemIndexDescriptor> systemIndexDescriptors = new ArrayList<>();
+        systemIndexDescriptors.add(new SystemIndexDescriptor(".test", "test"));
+        systemIndexDescriptors.add(new SystemIndexDescriptor(".test3", "test"));
+        systemIndexDescriptors.add(new SystemIndexDescriptor(".pattern-test*", "test-1"));
+        systemIndexDescriptors.add(new SystemIndexDescriptor(".pattern-test-overlapping", "test-2"));
+
+        ThreadPool testThreadPool = new TestThreadPool(getTestName());
+        try {
+            MetaDataCreateIndexService checkerService = new MetaDataCreateIndexService(
+                Settings.EMPTY,
+                ClusterServiceUtils.createClusterService(testThreadPool),
+                null,
+                null,
+                null,
+                null,
+                null,
+                testThreadPool,
+                null,
+                systemIndexDescriptors,
+                false
+            );
+            // Check deprecations
+            checkerService.validateIndexName(".test2", ClusterState.EMPTY_STATE, false);
+            assertWarnings("index name [.test2] starts with a dot '.', in the next major version, index " +
+                "names starting with a dot are reserved for hidden indices and system indices");
+
+            // Check non-system hidden indices don't trigger a warning
+            checkerService.validateIndexName(".test2", ClusterState.EMPTY_STATE, true);
+
+            // Check NO deprecation warnings if we give the index name
+            checkerService.validateIndexName(".test", ClusterState.EMPTY_STATE, false);
+            checkerService.validateIndexName(".test3", ClusterState.EMPTY_STATE, false);
+
+            // Check that patterns with wildcards work
+            checkerService.validateIndexName(".pattern-test", ClusterState.EMPTY_STATE, false);
+            checkerService.validateIndexName(".pattern-test-with-suffix", ClusterState.EMPTY_STATE, false);
+            checkerService.validateIndexName(".pattern-test-other-suffix", ClusterState.EMPTY_STATE, false);
+
+            // Check that an exception is thrown if more than one descriptor matches the index name
+            AssertionError exception = expectThrows(AssertionError.class,
+                () -> checkerService.validateIndexName(".pattern-test-overlapping", ClusterState.EMPTY_STATE, false));
+            assertThat(exception.getMessage(),
+                containsString("index name [.pattern-test-overlapping] is claimed as a system index by multiple system index patterns:"));
+            assertThat(exception.getMessage(), containsString("pattern: [.pattern-test*], description: [test-1]"));
+            assertThat(exception.getMessage(), containsString("pattern: [.pattern-test-overlapping], description: [test-2]"));
+
+        } finally {
+            testThreadPool.shutdown();
+        }
+    }
+
     public void testParseMappingsAppliesDataFromTemplateAndRequest() throws Exception {
         IndexTemplateMetaData templateMetaData = addMatchingTemplate(templateBuilder -> {
             templateBuilder.putAlias(AliasMetaData.builder("alias1"));

+ 117 - 0
server/src/test/java/org/elasticsearch/indices/SystemIndexDescriptorTests.java

@@ -0,0 +1,117 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.indices;
+
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+
+public class SystemIndexDescriptorTests extends ESTestCase {
+
+    public void testValidation() {
+        {
+            Exception ex = expectThrows(NullPointerException.class,
+                () -> new SystemIndexDescriptor(null, randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must not be null"));
+        }
+
+        {
+            Exception ex = expectThrows(IllegalArgumentException.class,
+                () -> new SystemIndexDescriptor("", randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must at least 2 characters in length"));
+        }
+
+        {
+            Exception ex = expectThrows(IllegalArgumentException.class,
+                () -> new SystemIndexDescriptor(".", randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must at least 2 characters in length"));
+        }
+
+        {
+            Exception ex = expectThrows(IllegalArgumentException.class,
+                () -> new SystemIndexDescriptor(randomAlphaOfLength(10), randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must start with the character [.]"));
+        }
+
+        {
+            Exception ex = expectThrows(IllegalArgumentException.class,
+                () -> new SystemIndexDescriptor(".*", randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must not start with the character sequence [.*] to prevent conflicts"));
+        }
+        {
+            Exception ex = expectThrows(IllegalArgumentException.class,
+                () -> new SystemIndexDescriptor(".*" + randomAlphaOfLength(10), randomAlphaOfLength(5)));
+            assertThat(ex.getMessage(), containsString("must not start with the character sequence [.*] to prevent conflicts"));
+        }
+    }
+
+    public void testBasicOverlappingPatterns() {
+        SystemIndexDescriptor broadPattern = new SystemIndexDescriptor(".a*c*", "test");
+        SystemIndexDescriptor notOverlapping = new SystemIndexDescriptor(".bbbddd*", "test");
+        SystemIndexDescriptor overlapping1 = new SystemIndexDescriptor(".ac*", "test");
+        SystemIndexDescriptor overlapping2 = new SystemIndexDescriptor(".aaaabbbccc", "test");
+        SystemIndexDescriptor overlapping3 = new SystemIndexDescriptor(".aaabb*cccddd*", "test");
+
+        // These sources have fixed prefixes to make sure they sort in the same order, so that the error message is consistent
+        // across tests
+        String broadPatternSource = "AAA" + randomAlphaOfLength(5);
+        String otherSource = "ZZZ" + randomAlphaOfLength(6);
+        Map<String, Collection<SystemIndexDescriptor>> descriptors = new HashMap<>();
+        descriptors.put(broadPatternSource, Arrays.asList(broadPattern));
+        descriptors.put(otherSource, Arrays.asList(notOverlapping, overlapping1, overlapping2, overlapping3));
+
+        IllegalStateException exception = expectThrows(IllegalStateException.class,
+            () -> SystemIndexDescriptor.checkForOverlappingPatterns(descriptors));
+        assertThat(exception.getMessage(), containsString("a system index descriptor [" + broadPattern +
+            "] from plugin [" + broadPatternSource + "] overlaps with other system index descriptors:"));
+        String fromPluginString = " from plugin [" + otherSource + "]";
+        assertThat(exception.getMessage(), containsString(overlapping1.toString() + fromPluginString));
+        assertThat(exception.getMessage(), containsString(overlapping2.toString() + fromPluginString));
+        assertThat(exception.getMessage(), containsString(overlapping3.toString() + fromPluginString));
+        assertThat(exception.getMessage(), not(containsString(notOverlapping.toString())));
+    }
+
+    public void testComplexOverlappingPatterns() {
+        // These patterns are slightly more complex to detect because pattern1 does not match pattern2 and vice versa
+        SystemIndexDescriptor pattern1 = new SystemIndexDescriptor(".a*c", "test");
+        SystemIndexDescriptor pattern2 = new SystemIndexDescriptor(".ab*", "test");
+
+        // These sources have fixed prefixes to make sure they sort in the same order, so that the error message is consistent
+        // across tests
+        String source1 = "AAA" + randomAlphaOfLength(5);
+        String source2 = "ZZZ" + randomAlphaOfLength(6);
+        Map<String, Collection<SystemIndexDescriptor>> descriptors = new HashMap<>();
+        descriptors.put(source1, Arrays.asList(pattern1));
+        descriptors.put(source2, Arrays.asList(pattern2));
+
+        IllegalStateException exception = expectThrows(IllegalStateException.class,
+            () -> SystemIndexDescriptor.checkForOverlappingPatterns(descriptors));
+        assertThat(exception.getMessage(), containsString("a system index descriptor [" + pattern1 +
+            "] from plugin [" + source1 + "] overlaps with other system index descriptors:"));
+        assertThat(exception.getMessage(), containsString(pattern2.toString() + " from plugin [" + source2 + "]"));
+
+    }
+}

+ 1 - 1
server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java

@@ -205,7 +205,7 @@ public class ClusterStateChanges {
             allocationService, IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, indicesService, threadPool);
         MetaDataCreateIndexService createIndexService = new MetaDataCreateIndexService(SETTINGS, clusterService, indicesService,
             allocationService, new AliasValidator(), environment,
-            IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, threadPool, xContentRegistry, true);
+            IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, threadPool, xContentRegistry, Collections.emptyList(), true);
 
         transportCloseIndexAction = new TransportCloseIndexAction(SETTINGS, transportService, clusterService, threadPool,
             indexStateService, clusterSettings, actionFilters, indexNameExpressionResolver, destructiveOperations);

+ 1 - 1
server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java

@@ -1230,7 +1230,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
                 final MetaDataCreateIndexService metaDataCreateIndexService = new MetaDataCreateIndexService(settings, clusterService,
                     indicesService,
                     allocationService, new AliasValidator(), environment, indexScopedSettings,
-                    threadPool, namedXContentRegistry, false);
+                    threadPool, namedXContentRegistry, Collections.emptyList(), false);
                 actions.put(CreateIndexAction.INSTANCE,
                     new TransportCreateIndexAction(
                         transportService, clusterService, threadPool,

+ 1 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/enrich/EnrichPolicy.java

@@ -33,6 +33,7 @@ import java.util.Objects;
 public final class EnrichPolicy implements Writeable, ToXContentFragment {
 
     public static final String ENRICH_INDEX_NAME_BASE = ".enrich-";
+    public static final String ENRICH_INDEX_PATTERN = ENRICH_INDEX_NAME_BASE + "*";
 
     public static final String MATCH_TYPE = "match";
     public static final String GEO_MATCH_TYPE = "geo_match";

+ 1 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/annotations/AnnotationIndex.java

@@ -38,6 +38,7 @@ public class AnnotationIndex {
     public static final String WRITE_ALIAS_NAME = ".ml-annotations-write";
     // Exposed for testing, but always use the aliases in non-test code
     public static final String INDEX_NAME = ".ml-annotations-6";
+    public static final String INDEX_PATTERN = ".ml-annotations*";
 
     /**
      * Create the .ml-annotations index with correct mappings if it does not already

+ 5 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java

@@ -8,8 +8,13 @@ package org.elasticsearch.xpack.core.ml.job.persistence;
 public final class AnomalyDetectorsIndexFields {
 
     public static final String CONFIG_INDEX = ".ml-config";
+
     public static final String RESULTS_INDEX_PREFIX = ".ml-anomalies-";
+    public static final String RESULTS_INDEX_PATTERN = RESULTS_INDEX_PREFIX + "*";
+
     public static final String STATE_INDEX_PREFIX = ".ml-state";
+    public static final String STATE_INDEX_PATTERN = STATE_INDEX_PREFIX + "*";
+
     public static final String RESULTS_INDEX_DEFAULT = "shared";
 
     private AnomalyDetectorsIndexFields() {}

+ 12 - 2
x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPlugin.java

@@ -24,11 +24,12 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.ingest.Processor;
 import org.elasticsearch.license.XPackLicenseState;
-import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.IngestPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.script.ScriptService;
@@ -57,14 +58,16 @@ import org.elasticsearch.xpack.enrich.rest.RestGetEnrichPolicyAction;
 import org.elasticsearch.xpack.enrich.rest.RestPutEnrichPolicyAction;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
 import static org.elasticsearch.xpack.core.XPackSettings.ENRICH_ENABLED_SETTING;
+import static org.elasticsearch.xpack.core.enrich.EnrichPolicy.ENRICH_INDEX_PATTERN;
 
-public class EnrichPlugin extends Plugin implements ActionPlugin, IngestPlugin {
+public class EnrichPlugin extends Plugin implements SystemIndexPlugin, IngestPlugin {
 
     static final Setting<Integer> ENRICH_FETCH_SIZE_SETTING = Setting.intSetting(
         "enrich.fetch_size",
@@ -240,4 +243,11 @@ public class EnrichPlugin extends Plugin implements ActionPlugin, IngestPlugin {
             ENRICH_MAX_FORCE_MERGE_ATTEMPTS
         );
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.singletonList(
+            new SystemIndexDescriptor(ENRICH_INDEX_PATTERN, "Contains data to support enrich ingest processors.")
+        );
+    }
 }

+ 12 - 3
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

@@ -28,8 +28,9 @@ import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.IndexModule;
-import org.elasticsearch.plugins.ActionPlugin;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.script.ScriptService;
@@ -50,9 +51,9 @@ import org.elasticsearch.xpack.core.ilm.ReadOnlyAction;
 import org.elasticsearch.xpack.core.ilm.RolloverAction;
 import org.elasticsearch.xpack.core.ilm.SetPriorityAction;
 import org.elasticsearch.xpack.core.ilm.ShrinkAction;
-import org.elasticsearch.xpack.core.ilm.WaitForSnapshotAction;
 import org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType;
 import org.elasticsearch.xpack.core.ilm.UnfollowAction;
+import org.elasticsearch.xpack.core.ilm.WaitForSnapshotAction;
 import org.elasticsearch.xpack.core.ilm.action.DeleteLifecycleAction;
 import org.elasticsearch.xpack.core.ilm.action.ExplainLifecycleAction;
 import org.elasticsearch.xpack.core.ilm.action.GetLifecycleAction;
@@ -133,7 +134,7 @@ import java.util.function.Supplier;
 
 import static org.elasticsearch.xpack.core.ClientHelper.INDEX_LIFECYCLE_ORIGIN;
 
-public class IndexLifecycle extends Plugin implements ActionPlugin {
+public class IndexLifecycle extends Plugin implements SystemIndexPlugin {
     private final SetOnce<IndexLifecycleService> indexLifecycleInitialisationService = new SetOnce<>();
     private final SetOnce<ILMHistoryStore> ilmHistoryStore = new SetOnce<>();
     private final SetOnce<SnapshotLifecycleService> snapshotLifecycleService = new SetOnce<>();
@@ -327,4 +328,12 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
             throw new ElasticsearchException("unable to close index lifecycle services", e);
         }
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        //TODO: The SLM history store should be non-dot-prefixed hidden indices, but need to be here for now
+        // to prevent warnings
+        return Collections.singletonList(new SystemIndexDescriptor(".slm-history-*",
+            "Contains a history of Snapshot Lifecycle Management operations"));
+    }
 }

+ 12 - 2
x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/Logstash.java

@@ -12,14 +12,17 @@ import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.plugins.ActionPlugin;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.xpack.core.XPackSettings;
 import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction;
 import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
 import org.elasticsearch.xpack.core.template.TemplateUtils;
 
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.function.UnaryOperator;
@@ -28,8 +31,9 @@ import java.util.regex.Pattern;
 /**
  * This class supplies the logstash featureset and templates
  */
-public class Logstash extends Plugin implements ActionPlugin {
+public class Logstash extends Plugin implements SystemIndexPlugin {
 
+    private static final String LOGSTASH_CONCRETE_INDEX_NAME = ".logstash";
     private static final String LOGSTASH_TEMPLATE_FILE_NAME = "logstash-management";
     private static final String LOGSTASH_INDEX_TEMPLATE_NAME = ".logstash-management";
     private static final String OLD_LOGSTASH_INDEX_NAME = "logstash-index-template";
@@ -59,4 +63,10 @@ public class Logstash extends Plugin implements ActionPlugin {
             return templates;
         };
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.singletonList(new SystemIndexDescriptor(LOGSTASH_CONCRETE_INDEX_NAME,
+            "Contains data for Logstash Central Management"));
+    }
 }

+ 21 - 5
x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java

@@ -42,17 +42,18 @@ import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.analysis.TokenizerFactory;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
 import org.elasticsearch.ingest.Processor;
 import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.monitor.os.OsProbe;
 import org.elasticsearch.monitor.os.OsStats;
 import org.elasticsearch.persistent.PersistentTasksExecutor;
-import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.AnalysisPlugin;
 import org.elasticsearch.plugins.IngestPlugin;
 import org.elasticsearch.plugins.PersistentTaskPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.script.ScriptService;
@@ -68,7 +69,6 @@ import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
 import org.elasticsearch.xpack.core.ml.MachineLearningField;
 import org.elasticsearch.xpack.core.ml.MlMetaIndex;
 import org.elasticsearch.xpack.core.ml.action.CloseJobAction;
-import org.elasticsearch.xpack.core.ml.action.ExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.core.ml.action.DeleteCalendarAction;
 import org.elasticsearch.xpack.core.ml.action.DeleteCalendarEventAction;
 import org.elasticsearch.xpack.core.ml.action.DeleteDataFrameAnalyticsAction;
@@ -80,6 +80,7 @@ import org.elasticsearch.xpack.core.ml.action.DeleteJobAction;
 import org.elasticsearch.xpack.core.ml.action.DeleteModelSnapshotAction;
 import org.elasticsearch.xpack.core.ml.action.DeleteTrainedModelAction;
 import org.elasticsearch.xpack.core.ml.action.EvaluateDataFrameAction;
+import org.elasticsearch.xpack.core.ml.action.ExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.core.ml.action.FinalizeJobExecutionAction;
 import org.elasticsearch.xpack.core.ml.action.FindFileStructureAction;
 import org.elasticsearch.xpack.core.ml.action.FlushJobAction;
@@ -130,6 +131,7 @@ import org.elasticsearch.xpack.core.ml.action.UpdateModelSnapshotAction;
 import org.elasticsearch.xpack.core.ml.action.UpdateProcessAction;
 import org.elasticsearch.xpack.core.ml.action.ValidateDetectorAction;
 import org.elasticsearch.xpack.core.ml.action.ValidateJobConfigAction;
+import org.elasticsearch.xpack.core.ml.annotations.AnnotationIndex;
 import org.elasticsearch.xpack.core.ml.dataframe.analyses.MlDataFrameAnalysisNamedXContentProvider;
 import org.elasticsearch.xpack.core.ml.dataframe.evaluation.MlEvaluationNamedXContentProvider;
 import org.elasticsearch.xpack.core.ml.inference.MlInferenceNamedXContentProvider;
@@ -140,7 +142,6 @@ import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings;
 import org.elasticsearch.xpack.core.ml.notifications.AuditorField;
 import org.elasticsearch.xpack.core.template.TemplateUtils;
 import org.elasticsearch.xpack.ml.action.TransportCloseJobAction;
-import org.elasticsearch.xpack.ml.action.TransportExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.action.TransportDeleteCalendarAction;
 import org.elasticsearch.xpack.ml.action.TransportDeleteCalendarEventAction;
 import org.elasticsearch.xpack.ml.action.TransportDeleteDataFrameAnalyticsAction;
@@ -152,6 +153,7 @@ import org.elasticsearch.xpack.ml.action.TransportDeleteJobAction;
 import org.elasticsearch.xpack.ml.action.TransportDeleteModelSnapshotAction;
 import org.elasticsearch.xpack.ml.action.TransportDeleteTrainedModelAction;
 import org.elasticsearch.xpack.ml.action.TransportEvaluateDataFrameAction;
+import org.elasticsearch.xpack.ml.action.TransportExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.action.TransportFinalizeJobExecutionAction;
 import org.elasticsearch.xpack.ml.action.TransportFindFileStructureAction;
 import org.elasticsearch.xpack.ml.action.TransportFlushJobAction;
@@ -266,9 +268,9 @@ import org.elasticsearch.xpack.ml.rest.datafeeds.RestPutDatafeedAction;
 import org.elasticsearch.xpack.ml.rest.datafeeds.RestStartDatafeedAction;
 import org.elasticsearch.xpack.ml.rest.datafeeds.RestStopDatafeedAction;
 import org.elasticsearch.xpack.ml.rest.datafeeds.RestUpdateDatafeedAction;
-import org.elasticsearch.xpack.ml.rest.dataframe.RestExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.rest.dataframe.RestDeleteDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.rest.dataframe.RestEvaluateDataFrameAction;
+import org.elasticsearch.xpack.ml.rest.dataframe.RestExplainDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.rest.dataframe.RestGetDataFrameAnalyticsAction;
 import org.elasticsearch.xpack.ml.rest.dataframe.RestGetDataFrameAnalyticsStatsAction;
 import org.elasticsearch.xpack.ml.rest.dataframe.RestPutDataFrameAnalyticsAction;
@@ -323,7 +325,7 @@ import java.util.function.UnaryOperator;
 import static java.util.Collections.emptyList;
 import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME;
 
-public class MachineLearning extends Plugin implements ActionPlugin, AnalysisPlugin, IngestPlugin, PersistentTaskPlugin {
+public class MachineLearning extends Plugin implements SystemIndexPlugin, AnalysisPlugin, IngestPlugin, PersistentTaskPlugin {
     public static final String NAME = "ml";
     public static final String BASE_PATH = "/_ml/";
     public static final String PRE_V7_BASE_PATH = "/_xpack/ml/";
@@ -1044,4 +1046,18 @@ public class MachineLearning extends Plugin implements ActionPlugin, AnalysisPlu
         return namedXContent;
     }
 
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        //TODO: Some of the below should be converted to non-dot-prefixed hidden indices. I am not sure which ones.
+        // Either way, they should be here until then to prevent deprecation warnings
+        return Collections.unmodifiableList(Arrays.asList(
+            new SystemIndexDescriptor(MlMetaIndex.INDEX_NAME, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(AnomalyDetectorsIndexFields.STATE_INDEX_PATTERN, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(AnomalyDetectorsIndexFields.CONFIG_INDEX, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(AnomalyDetectorsIndexFields.RESULTS_INDEX_PATTERN, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(InferenceIndexConstants.INDEX_PATTERN, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(".ml-notifications-*", this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(AnnotationIndex.INDEX_PATTERN, this.getClass().getSimpleName())
+        ));
+    }
 }

+ 16 - 2
x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java

@@ -42,12 +42,12 @@ import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.http.HttpServerTransport;
 import org.elasticsearch.index.IndexModule;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
 import org.elasticsearch.ingest.Processor;
 import org.elasticsearch.license.License;
 import org.elasticsearch.license.LicenseService;
 import org.elasticsearch.license.XPackLicenseState;
-import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.ClusterPlugin;
 import org.elasticsearch.plugins.DiscoveryPlugin;
 import org.elasticsearch.plugins.ExtensiblePlugin;
@@ -55,6 +55,7 @@ import org.elasticsearch.plugins.IngestPlugin;
 import org.elasticsearch.plugins.MapperPlugin;
 import org.elasticsearch.plugins.NetworkPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.rest.RestHeaderDefinition;
@@ -128,6 +129,7 @@ import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissions;
 import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissionsCache;
 import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore;
 import org.elasticsearch.xpack.core.security.authz.store.RoleRetrievalResult;
+import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames;
 import org.elasticsearch.xpack.core.security.support.Automatons;
 import org.elasticsearch.xpack.core.security.user.AnonymousUser;
 import org.elasticsearch.xpack.core.ssl.SSLConfiguration;
@@ -272,7 +274,7 @@ import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames
 import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT;
 import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7;
 
-public class Security extends Plugin implements ActionPlugin, IngestPlugin, NetworkPlugin, ClusterPlugin,
+public class Security extends Plugin implements SystemIndexPlugin, IngestPlugin, NetworkPlugin, ClusterPlugin,
         DiscoveryPlugin, MapperPlugin, ExtensiblePlugin {
 
     private static final Logger logger = LogManager.getLogger(Security.class);
@@ -1064,4 +1066,16 @@ public class Security extends Plugin implements ActionPlugin, IngestPlugin, Netw
             return groupFactory.get();
          }
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.unmodifiableList(Arrays.asList(
+            new SystemIndexDescriptor(SECURITY_MAIN_ALIAS, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7, this.getClass().getSimpleName()),
+
+            new SystemIndexDescriptor(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(RestrictedIndicesNames.INTERNAL_SECURITY_TOKENS_INDEX_7, this.getClass().getSimpleName())
+            ));
+    }
 }

+ 5 - 5
x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml

@@ -272,7 +272,7 @@ setup:
       headers:
         Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
       indices.create:
-        index: ".ml-special-index-to-avoid-picking-up-template"
+        index: "ml-special-index-to-avoid-picking-up-template"
         body:
           mappings:
             properties:
@@ -283,28 +283,28 @@ setup:
       headers:
         Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
       indices.put_alias:
-        index: ".ml-special-index-to-avoid-picking-up-template"
+        index: "ml-special-index-to-avoid-picking-up-template"
         name: ".ml-anomalies-get-datafeed-stats-1"
 
   - do:
       headers:
         Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
       indices.put_alias:
-        index: ".ml-special-index-to-avoid-picking-up-template"
+        index: "ml-special-index-to-avoid-picking-up-template"
         name: ".ml-anomalies-.write-get-datafeed-stats-1"
 
   - do:
       headers:
         Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
       indices.put_alias:
-        index: ".ml-special-index-to-avoid-picking-up-template"
+        index: "ml-special-index-to-avoid-picking-up-template"
         name: ".ml-anomalies-get-datafeed-stats-2"
 
   - do:
       headers:
         Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
       indices.put_alias:
-        index: ".ml-special-index-to-avoid-picking-up-template"
+        index: "ml-special-index-to-avoid-picking-up-template"
         name: ".ml-anomalies-.write-get-datafeed-stats-2"
 
   - do:

+ 13 - 2
x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java

@@ -27,11 +27,12 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry.Entry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.persistent.PersistentTasksExecutor;
-import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.PersistentTaskPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.script.ScriptService;
@@ -113,7 +114,7 @@ import java.util.function.UnaryOperator;
 
 import static java.util.Collections.emptyList;
 
-public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlugin {
+public class Transform extends Plugin implements SystemIndexPlugin, PersistentTaskPlugin {
 
     public static final String NAME = "transform";
     public static final String TASK_THREAD_POOL_NAME = "transform_indexing";
@@ -313,4 +314,14 @@ public class Transform extends Plugin implements ActionPlugin, PersistentTaskPlu
     public List<Entry> getNamedXContent() {
         return new TransformNamedXContentProvider().getNamedXContentParsers();
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.unmodifiableList(Arrays.asList(
+            // TODO: Verify that these should be system indices, rather than hidden indices
+            new SystemIndexDescriptor(TransformInternalIndexConstants.INDEX_NAME_PATTERN, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(TransformInternalIndexConstants.AUDIT_INDEX_PATTERN, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(TransformInternalIndexConstants.AUDIT_INDEX_PATTERN_DEPRECATED, this.getClass().getSimpleName())
+        ));
+    }
 }

+ 15 - 2
x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java

@@ -39,12 +39,13 @@ import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.IndexModule;
+import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.node.Node;
-import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.plugins.ReloadablePlugin;
 import org.elasticsearch.plugins.ScriptPlugin;
+import org.elasticsearch.plugins.SystemIndexPlugin;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
 import org.elasticsearch.script.ScriptContext;
@@ -197,7 +198,7 @@ import static java.util.Collections.emptyList;
 import static org.elasticsearch.common.settings.Setting.Property.NodeScope;
 import static org.elasticsearch.xpack.core.ClientHelper.WATCHER_ORIGIN;
 
-public class Watcher extends Plugin implements ActionPlugin, ScriptPlugin, ReloadablePlugin {
+public class Watcher extends Plugin implements SystemIndexPlugin, ScriptPlugin, ReloadablePlugin {
 
     // This setting is only here for backward compatibility reasons as 6.x indices made use of it. It can be removed in 8.x.
     @Deprecated
@@ -682,4 +683,16 @@ public class Watcher extends Plugin implements ActionPlugin, ScriptPlugin, Reloa
         }
         reloadableServices.forEach(s -> s.reload(settings));
     }
+
+    @Override
+    public Collection<SystemIndexDescriptor> getSystemIndexDescriptors() {
+        return Collections.unmodifiableList(Arrays.asList(
+            new SystemIndexDescriptor(Watch.INDEX, this.getClass().getSimpleName()),
+            new SystemIndexDescriptor(TriggeredWatchStoreField.INDEX_NAME, this.getClass().getSimpleName()),
+
+            //TODO: The below should be converted to hidden indices, but need to be included here until then to prevent
+            // deprecation warnings
+            new SystemIndexDescriptor(".watch-history-*", this.getClass().getSimpleName())
+        ));
+    }
 }