Bläddra i källkod

Disallow composable index template that have both data stream and aliases definitions (#67886)

Index aliases are not allowed to refer to backing indices of data streams.
Adding an alias that points to a backing index results into a validation error.

However when defining aliases and a data stream definition on an index template,
it is still possible to for aliases that refer to backing indices to be created
when a data stream or new backing index is created.

This change add additional validation that prevents defining aliases and data
stream definition together in a composable index template or component templates
that are referred by an composable index template. This should fix the mentioned
bug.

This checks only enables this validation when adding/updating a composable index
and component template. There may be templates in the wild that have both
data stream and aliases definitions, so we currently can't fail cluster states
that contain aliases that refer to backing indices. So instead a warning header
is emitted.

Closes #67730
Martijn van Groningen 4 år sedan
förälder
incheckning
06c213f20e

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

@@ -314,7 +314,7 @@ public class MetadataRolloverService {
 
         final String matchedV2Template = findV2Template(metadata, rolloverIndexName, isHidden == null ? false : isHidden);
         if (matchedV2Template != null) {
-            List<Map<String, AliasMetadata>> aliases = MetadataIndexTemplateService.resolveAliases(metadata, matchedV2Template);
+            List<Map<String, AliasMetadata>> aliases = MetadataIndexTemplateService.resolveAliases(metadata, matchedV2Template, false);
             for (Map<String, AliasMetadata> aliasConfig : aliases) {
                 if (aliasConfig.containsKey(rolloverRequestAlias)) {
                     throw new IllegalArgumentException(String.format(Locale.ROOT,

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java

@@ -161,7 +161,7 @@ public class TransportSimulateIndexTemplateAction
         Settings settings = resolveSettings(simulatedState.metadata(), matchingTemplate);
 
         List<Map<String, AliasMetadata>> resolvedAliases = MetadataIndexTemplateService.resolveAliases(simulatedState.metadata(),
-            matchingTemplate);
+            matchingTemplate, true);
 
         // create the index with dummy settings in the cluster state so we can parse and validate the aliases
         Settings dummySettings = Settings.builder()

+ 26 - 0
server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java

@@ -43,6 +43,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
+import org.elasticsearch.common.logging.HeaderWarning;
 import org.elasticsearch.common.regex.Regex;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
@@ -1449,6 +1450,31 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
                             " including '" + conflicts.iterator().next() + "'");
                     }
                 }
+
+                // Sanity check, because elsewhere a more user friendly error should have occurred:
+                List<String> conflictingAliases = indicesLookup.values().stream()
+                    .filter(ia -> ia.getType() == IndexAbstraction.Type.ALIAS)
+                    .filter(ia -> {
+                        for (IndexMetadata index : ia.getIndices()) {
+                            if (indicesLookup.get(index.getIndex().getName()).getParentDataStream() != null) {
+                                return true;
+                            }
+                        }
+
+                        return false;
+                    })
+                    .map(IndexAbstraction::getName)
+                    .collect(Collectors.toList());
+                if (conflictingAliases.isEmpty() == false) {
+                    // After backporting throw an IllegalStateException instead of logging a warning:
+                    // (in 7.x there might be aliases that refer to backing indices of a data stream and
+                    // throwing an exception here would avoid the cluster from functioning)
+                    String warning = "aliases " + conflictingAliases + " cannot refer to backing indices of data streams";
+                    // log as debug, this method is executed each time a new cluster state is created and could result
+                    // in many logs:
+                    logger.debug(warning);
+                    HeaderWarning.addWarning(warning);
+                }
             }
         }
 

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java

@@ -517,10 +517,10 @@ public class MetadataCreateIndexService {
 
         return applyCreateIndexWithTemporaryService(currentState, request, silent, null, tmpImd, mappings,
             indexService -> resolveAndValidateAliases(request.index(), request.aliases(),
-                MetadataIndexTemplateService.resolveAliases(currentState.metadata(), templateName), currentState.metadata(), aliasValidator,
+                MetadataIndexTemplateService.resolveAliases(currentState.metadata(), templateName, false), currentState.metadata(),
                 // the context is only used for validation so it's fine to pass fake values for the
                 // shard id and the current timestamp
-                xContentRegistry, indexService.newSearchExecutionContext(0, 0, null, () -> 0L, null, emptyMap())),
+                aliasValidator, xContentRegistry, indexService.newSearchExecutionContext(0, 0, null, () -> 0L, null, emptyMap())),
             Collections.singletonList(templateName), metadataTransformer);
     }
 

+ 28 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java

@@ -1043,8 +1043,21 @@ public class MetadataIndexTemplateService {
 
     /**
      * Resolve the given v2 template into an ordered list of aliases
+     *
+     * @param failIfTemplateHasDataStream Whether to skip validating if a template has a data stream definition and an alias definition.
+     *                                    This validation is needed so that no template gets created that creates datastream and also
+     *                                    a an alias pointing to the backing indices of a data stream. Unfortunately this validation
+     *                                    was missing in versions prior to 7.11, which mean that there are cluster states out there,
+     *                                    that have this malformed templates. This method is used when rolling over a data stream
+     *                                    or creating new data streams. In order for these clusters to avoid failing these operations
+     *                                    immediately after an upgrade the failure should be optional. So that there is time to change
+     *                                    these templates. The logic that adds/updates index and component templates shouldn't skip this
+     *                                    validation.
      */
-    public static List<Map<String, AliasMetadata>> resolveAliases(final Metadata metadata, final String templateName) {
+    public static List<Map<String, AliasMetadata>> resolveAliases(final Metadata metadata,
+                                                                  final String templateName,
+                                                                  // TODO: remove in master after backport to 7.x:
+                                                                  final boolean failIfTemplateHasDataStream) {
         final ComposableIndexTemplate template = metadata.templatesV2().get(templateName);
         assert template != null : "attempted to resolve aliases for a template [" + templateName +
             "] that did not exist in the cluster state";
@@ -1064,6 +1077,19 @@ public class MetadataIndexTemplateService {
         Optional.ofNullable(template.template())
             .map(Template::aliases)
             .ifPresent(aliases::add);
+
+        // A template that creates data streams can't also create aliases.
+        // (otherwise we end up with aliases pointing to backing indices of data streams)
+        if (aliases.size() > 0 && template.getDataStreamTemplate() != null) {
+            if (failIfTemplateHasDataStream) {
+                throw new IllegalArgumentException("template [" + templateName + "] has alias and data stream definitions");
+            } else {
+                String warning = "template [" + templateName + "] has alias and data stream definitions";
+                logger.warn(warning);
+                HeaderWarning.addWarning(warning);
+            }
+        }
+
         // Aliases are applied in order, but subsequent alias configuration from the same name is
         // ignored, so in order for the order to be correct, alias configuration should be in order
         // of precedence (with the index template first)
@@ -1115,7 +1141,7 @@ public class MetadataIndexTemplateService {
             tempIndexService -> {
                 // Validate aliases
                 MetadataCreateIndexService.resolveAndValidateAliases(temporaryIndexName, Collections.emptySet(),
-                    MetadataIndexTemplateService.resolveAliases(stateWithIndex.metadata(), templateName), stateWithIndex.metadata(),
+                    MetadataIndexTemplateService.resolveAliases(stateWithIndex.metadata(), templateName, true), stateWithIndex.metadata(),
                     new AliasValidator(),
                     // the context is only used for validation so it's fine to pass fake values for the
                     // shard id and the current timestamp

+ 86 - 0
server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java

@@ -631,6 +631,92 @@ public class MetadataRolloverServiceTests extends ESTestCase {
         }
     }
 
+    public void testRolloverDataStreamWorksWithTemplateThatAlsoCreatesAliases() throws Exception {
+        final DataStream dataStream = DataStreamTestHelper.randomInstance()
+            // ensure no replicate data stream
+            .promoteDataStream();
+        ComposableIndexTemplate template = new ComposableIndexTemplate.Builder().indexPatterns(List.of(dataStream.getName() + "*"))
+            .template(new Template(null, null, Map.of("my-alias", AliasMetadata.newAliasMetadataBuilder("my-alias").build())))
+            .dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate()).build();
+        Metadata.Builder builder = Metadata.builder();
+        builder.put("template", template);
+        for (Index index : dataStream.getIndices()) {
+            builder.put(DataStreamTestHelper.getIndexMetadataBuilderForIndex(index));
+        }
+        builder.put(dataStream);
+        final ClusterState clusterState = ClusterState.builder(new ClusterName("test")).metadata(builder).build();
+
+        ThreadPool testThreadPool = new TestThreadPool(getTestName());
+        try {
+            DateFieldMapper dateFieldMapper
+                = new DateFieldMapper.Builder("@timestamp", DateFieldMapper.Resolution.MILLISECONDS, null, false, Version.CURRENT)
+                .build(new ContentPath());
+            MappedFieldType mockedTimestampFieldType = mock(MappedFieldType.class);
+            when(mockedTimestampFieldType.name()).thenReturn("_data_stream_timestamp");
+            MetadataFieldMapper mockedTimestampField = new MetadataFieldMapper(mockedTimestampFieldType) {
+                @Override
+                protected String contentType() {
+                    return null;
+                }
+            };
+            MappingLookup mappingLookup = new MappingLookup(
+                Mapping.EMPTY,
+                List.of(mockedTimestampField, dateFieldMapper),
+                List.of(),
+                List.of(),
+                null,
+                null,
+                null);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool);
+            Environment env = mock(Environment.class);
+            when(env.sharedDataFile()).thenReturn(null);
+            AllocationService allocationService = mock(AllocationService.class);
+            when(allocationService.reroute(any(ClusterState.class), any(String.class))).then(i -> i.getArguments()[0]);
+            DocumentMapper documentMapper = mock(DocumentMapper.class);
+            when(documentMapper.mappers()).thenReturn(mappingLookup);
+            when(documentMapper.type()).thenReturn("_doc");
+            CompressedXContent mapping =
+                new CompressedXContent("{\"_doc\":" + generateMapping(dataStream.getTimeStampField().getName(), "date") + "}");
+            when(documentMapper.mappingSource()).thenReturn(mapping);
+            RoutingFieldMapper routingFieldMapper = mock(RoutingFieldMapper.class);
+            when(routingFieldMapper.required()).thenReturn(false);
+            when(documentMapper.routingFieldMapper()).thenReturn(routingFieldMapper);
+            IndicesService indicesService = mockIndicesServices(documentMapper);
+            IndexNameExpressionResolver mockIndexNameExpressionResolver = mock(IndexNameExpressionResolver.class);
+            when(mockIndexNameExpressionResolver.resolveDateMathExpression(any())).then(returnsFirstArg());
+
+            ShardLimitValidator shardLimitValidator = new ShardLimitValidator(Settings.EMPTY, clusterService);
+            MetadataCreateIndexService createIndexService = new MetadataCreateIndexService(Settings.EMPTY,
+                clusterService, indicesService, allocationService, new AliasValidator(), shardLimitValidator, env,
+                IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, testThreadPool, null, new SystemIndices(Map.of()), false);
+            MetadataIndexAliasesService indexAliasesService = new MetadataIndexAliasesService(clusterService, indicesService,
+                new AliasValidator(), null, xContentRegistry());
+            MetadataRolloverService rolloverService = new MetadataRolloverService(testThreadPool, createIndexService, indexAliasesService,
+                mockIndexNameExpressionResolver);
+
+            MaxDocsCondition condition = new MaxDocsCondition(randomNonNegativeLong());
+            List<Condition<?>> metConditions = Collections.singletonList(condition);
+            CreateIndexRequest createIndexRequest = new CreateIndexRequest("_na_");
+
+            // Ensure that a warning header is emitted
+            MetadataRolloverService.RolloverResult rolloverResult =
+                rolloverService.rolloverClusterState(clusterState, dataStream.getName(), null, createIndexRequest, metConditions,
+                    randomBoolean(), false);
+            assertWarnings(
+                "aliases [my-alias] cannot refer to backing indices of data streams",
+                "template [template] has alias and data stream definitions"
+            );
+
+            // Just checking that the rollover was successful:
+            String sourceIndexName = DataStream.getDefaultBackingIndexName(dataStream.getName(), dataStream.getGeneration());
+            String newIndexName = DataStream.getDefaultBackingIndexName(dataStream.getName(), dataStream.getGeneration() + 1);
+            assertEquals(sourceIndexName, rolloverResult.sourceIndexName);
+            assertEquals(newIndexName, rolloverResult.rolloverIndexName);
+        } finally {
+            testThreadPool.shutdown();
+        }
+    }
+
     public void testValidation() throws Exception {
         final String rolloverTarget;
         final String sourceIndexName;

+ 1 - 1
server/src/test/java/org/elasticsearch/cluster/metadata/ComposableIndexTemplateTests.java

@@ -78,7 +78,7 @@ public class ComposableIndexTemplateTests extends AbstractDiffableSerializationT
             if (dataStreamTemplate != null || randomBoolean()) {
                 mappings = randomMappings(dataStreamTemplate);
             }
-            if (randomBoolean()) {
+            if (dataStreamTemplate == null && randomBoolean()) {
                 aliases = randomAliases();
             }
             template = new Template(settings, mappings, aliases);

+ 33 - 1
server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java

@@ -1114,12 +1114,44 @@ public class MetadataIndexTemplateServiceTests extends ESSingleNodeTestCase {
             List.of("ct_low", "ct_high"), 0L, 1L, null, null, null);
         state = service.addIndexTemplateV2(state, true, "my-template", it);
 
-        List<Map<String, AliasMetadata>> resolvedAliases = MetadataIndexTemplateService.resolveAliases(state.metadata(), "my-template");
+        List<Map<String, AliasMetadata>> resolvedAliases =
+            MetadataIndexTemplateService.resolveAliases(state.metadata(), "my-template", true);
 
         // These should be order of precedence, so the index template (a3), then ct_high (a1), then ct_low (a2)
         assertThat(resolvedAliases, equalTo(List.of(a3, a1, a2)));
     }
 
+    public void testResolveAliasesDataStreams() throws Exception {
+        Map<String, AliasMetadata> a1 = new HashMap<>();
+        a1.put("logs", AliasMetadata.newAliasMetadataBuilder("logs").build());
+
+        // index template can't have data streams and aliases
+        ComposableIndexTemplate it = new ComposableIndexTemplate(List.of("logs-*"),
+            new Template(null, null, a1), null, 0L, 1L, null, new ComposableIndexTemplate.DataStreamTemplate(), null);
+        ClusterState state1 = ClusterState.builder(ClusterState.EMPTY_STATE)
+            .metadata(Metadata.builder().put("1", it).build())
+            .build();
+        Exception e =
+            expectThrows(IllegalArgumentException.class, () -> MetadataIndexTemplateService.resolveAliases(state1.metadata(), "1", true));
+        assertThat(e.getMessage(), equalTo("template [1] has alias and data stream definitions"));
+        // Ignoring validation
+        assertThat(MetadataIndexTemplateService.resolveAliases(state1.metadata(), "1", false), equalTo(List.of(a1)));
+        assertWarnings("template [1] has alias and data stream definitions");
+
+        // index template can't have data streams and a component template with an aliases
+        ComponentTemplate componentTemplate = new ComponentTemplate(new Template(null, null, a1), null, null);
+        it = new ComposableIndexTemplate(List.of("logs-*"), null, List.of("c1"), 0L, 1L, null,
+            new ComposableIndexTemplate.DataStreamTemplate(), null);
+        ClusterState state2 = ClusterState.builder(ClusterState.EMPTY_STATE)
+            .metadata(Metadata.builder().put("1", it).put("c1", componentTemplate).build())
+            .build();
+        e = expectThrows(IllegalArgumentException.class, () -> MetadataIndexTemplateService.resolveAliases(state2.metadata(), "1", true));
+        assertThat(e.getMessage(), equalTo("template [1] has alias and data stream definitions"));
+        // Ignoring validation
+        assertThat(MetadataIndexTemplateService.resolveAliases(state2.metadata(), "1", false), equalTo(List.of(a1)));
+        assertWarnings("template [1] has alias and data stream definitions");
+    }
+
     public void testAddInvalidTemplate() throws Exception {
         ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList("a"), null,
             Arrays.asList("good", "bad"), null, null, null);

+ 11 - 0
x-pack/plugin/data-streams/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/datastreams/DataStreamsRestIT.java

@@ -8,6 +8,7 @@ package org.elasticsearch.xpack.datastreams;
 
 import org.elasticsearch.client.Request;
 import org.elasticsearch.client.Response;
+import org.elasticsearch.client.ResponseException;
 import org.elasticsearch.common.xcontent.support.XContentMapValues;
 import org.elasticsearch.test.rest.ESRestTestCase;
 import org.junit.After;
@@ -16,6 +17,8 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 
+import static org.hamcrest.Matchers.containsString;
+
 public class DataStreamsRestIT extends ESRestTestCase {
 
     @After
@@ -77,4 +80,12 @@ public class DataStreamsRestIT extends ESRestTestCase {
         Map<String, Object> results = entityAsMap(response);
         assertEquals(1, XContentMapValues.extractValue("hits.total.value", results));
     }
+
+    public void testAddingIndexTemplateWithAliasesAndDataStream() {
+        Request putComposableIndexTemplateRequest = new Request("PUT", "/_index_template/my-template");
+        String body = "{\"index_patterns\":[\"mypattern*\"],\"data_stream\":{},\"template\":{\"aliases\":{\"my-alias\":{}}}}";
+        putComposableIndexTemplateRequest.setJsonEntity(body);
+        Exception e = expectThrows(ResponseException.class, () -> client().performRequest(putComposableIndexTemplateRequest));
+        assertThat(e.getMessage(), containsString("template [my-template] has alias and data stream definitions"));
+    }
 }