Browse Source

The get aliases api should not return entries for data streams with no aliases (#72953)

The get alias api should take into account the aliases parameter when
returning aliases that refer to data streams and don't return entries
for data streams that don't have any aliases pointing to it.

Relates to #66163
Martijn van Groningen 4 years ago
parent
commit
4b2c3ab0b7

+ 0 - 3
docs/reference/indices/aliases.asciidoc

@@ -599,9 +599,6 @@ The get index alias API returns:
 [source,console-result]
 ----
 {
-  "logs-my_app-default": {
-    "aliases": {}
-  },
   "logs-nginx.access-prod": {
     "aliases": {
       "logs": {}

+ 8 - 5
server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java

@@ -73,7 +73,8 @@ public class TransportGetAliasesAction extends TransportMasterNodeReadAction<Get
         final SystemIndexAccessLevel systemIndexAccessLevel = indexNameExpressionResolver.getSystemIndexAccessLevel();
         ImmutableOpenMap<String, List<AliasMetadata>> aliases = state.metadata().findAliases(request, concreteIndices);
         listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state,
-            systemIndexAccessLevel, threadPool.getThreadContext(), systemIndices), postProcess(request, state)));
+            systemIndexAccessLevel, threadPool.getThreadContext(), systemIndices),
+            postProcess(indexNameExpressionResolver, request, state)));
     }
 
     /**
@@ -107,17 +108,19 @@ public class TransportGetAliasesAction extends TransportMasterNodeReadAction<Get
         return finalResponse;
     }
 
-    Map<String, List<DataStreamAlias>> postProcess(GetAliasesRequest request, ClusterState state) {
+    static Map<String, List<DataStreamAlias>> postProcess(IndexNameExpressionResolver resolver, GetAliasesRequest request,
+                                                          ClusterState state) {
         Map<String, List<DataStreamAlias>> result = new HashMap<>();
         boolean noAliasesSpecified = request.getOriginalAliases() == null || request.getOriginalAliases().length == 0;
-        List<String> requestedDataStreams =
-            indexNameExpressionResolver.dataStreamNames(state, request.indicesOptions(), request.indices());
+        List<String> requestedDataStreams = resolver.dataStreamNames(state, request.indicesOptions(), request.indices());
         for (String requestedDataStream : requestedDataStreams) {
             List<DataStreamAlias> aliases = state.metadata().dataStreamAliases().values().stream()
                 .filter(alias -> alias.getDataStreams().contains(requestedDataStream))
                 .filter(alias -> noAliasesSpecified || Regex.simpleMatch(request.aliases(), alias.getName()))
                 .collect(Collectors.toList());
-            result.put(requestedDataStream, aliases);
+            if (aliases.isEmpty() == false) {
+                result.put(requestedDataStream, aliases);
+            }
         }
         return result;
     }

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

@@ -1177,7 +1177,7 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
             return this;
         }
 
-        public boolean put(String name, String dataStream) {
+        public boolean put(String aliasName, String dataStream) {
             Map<String, DataStream> existingDataStream =
                 Optional.ofNullable((DataStreamMetadata) this.customs.get(DataStreamMetadata.TYPE))
                     .map(dsmd -> new HashMap<>(dsmd.dataStreams()))
@@ -1188,21 +1188,21 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
                     .orElse(new HashMap<>());
 
             if (existingDataStream.containsKey(dataStream) == false) {
-                throw new IllegalArgumentException("alias [" + name + "] refers to a non existing data stream [" + dataStream + "]");
+                throw new IllegalArgumentException("alias [" + aliasName + "] refers to a non existing data stream [" + dataStream + "]");
             }
 
-            DataStreamAlias alias = dataStreamAliases.get(name);
+            DataStreamAlias alias = dataStreamAliases.get(aliasName);
             if (alias == null) {
-                alias = new DataStreamAlias(name, List.of(dataStream));
+                alias = new DataStreamAlias(aliasName, List.of(dataStream));
             } else {
                 Set<String> dataStreams = new HashSet<>(alias.getDataStreams());
                 boolean added = dataStreams.add(dataStream);
                 if (added == false) {
                     return false;
                 }
-                alias = new DataStreamAlias(name, List.copyOf(dataStreams));
+                alias = new DataStreamAlias(aliasName, List.copyOf(dataStreams));
             }
-            dataStreamAliases.put(name, alias);
+            dataStreamAliases.put(aliasName, alias);
 
             this.customs.put(DataStreamMetadata.TYPE, new DataStreamMetadata(existingDataStream, dataStreamAliases));
             return true;

+ 37 - 0
server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java

@@ -10,20 +10,26 @@ package org.elasticsearch.action.admin.indices.alias.get;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.AliasMetadata;
+import org.elasticsearch.cluster.metadata.DataStreamAlias;
+import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.indices.EmptySystemIndices;
 import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.SystemIndices;
 import org.elasticsearch.indices.SystemIndices.SystemIndexAccessLevel;
+import org.elasticsearch.indices.TestIndexNameExpressionResolver;
 import org.elasticsearch.test.ESTestCase;
 
 import java.util.Collections;
 import java.util.List;
 
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 
 public class TransportGetAliasesActionTests extends ESTestCase {
@@ -184,6 +190,37 @@ public class TransportGetAliasesActionTests extends ESTestCase {
             "access to system indices and their aliases will not be allowed");
     }
 
+    public void testPostProcessDataStreamAliases() {
+        var resolver = TestIndexNameExpressionResolver.newInstance();
+        var tuples = List.of(new Tuple<>("logs-foo", 1), new Tuple<>("logs-bar", 1), new Tuple<>("logs-baz", 1));
+        var clusterState = DataStreamTestHelper.getClusterStateWithDataStreams(tuples, List.of());
+        var builder = Metadata.builder(clusterState.metadata());
+        builder.put("logs", "logs-foo");
+        builder.put("logs", "logs-bar");
+        builder.put("secret", "logs-bar");
+        clusterState = ClusterState.builder(clusterState).metadata(builder).build();
+
+        // return all all data streams with aliases
+        var getAliasesRequest = new GetAliasesRequest();
+        var result = TransportGetAliasesAction.postProcess(resolver, getAliasesRequest, clusterState);
+        assertThat(result.keySet(), containsInAnyOrder("logs-foo", "logs-bar"));
+        assertThat(result.get("logs-foo"), contains(new DataStreamAlias("logs", List.of("logs-bar", "logs-foo"))));
+        assertThat(result.get("logs-bar"), containsInAnyOrder(new DataStreamAlias("logs", List.of("logs-bar", "logs-foo")),
+            new DataStreamAlias("secret", List.of("logs-bar"))));
+
+        // filter by alias name
+        getAliasesRequest = new GetAliasesRequest("secret");
+        result = TransportGetAliasesAction.postProcess(resolver, getAliasesRequest, clusterState);
+        assertThat(result.keySet(), containsInAnyOrder("logs-bar"));
+        assertThat(result.get("logs-bar"), contains(new DataStreamAlias("secret", List.of("logs-bar"))));
+
+        // filter by data stream:
+        getAliasesRequest = new GetAliasesRequest().indices("logs-foo");
+        result = TransportGetAliasesAction.postProcess(resolver, getAliasesRequest, clusterState);
+        assertThat(result.keySet(), containsInAnyOrder("logs-foo"));
+        assertThat(result.get("logs-foo"), contains(new DataStreamAlias("logs", List.of("logs-bar", "logs-foo"))));
+    }
+
     public ClusterState systemIndexTestClusterState() {
         return ClusterState.builder(ClusterState.EMPTY_STATE)
             .metadata(Metadata.builder()

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

@@ -130,8 +130,7 @@ public class DataStreamsRestIT extends ESRestTestCase {
 
         getAliasesRequest = new Request("GET", "/_aliases");
         getAliasesResponse = entityAsMap(client().performRequest(getAliasesRequest));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-myapp1.aliases", getAliasesResponse));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-myapp2.aliases", getAliasesResponse));
+        assertEquals(Map.of(), getAliasesResponse);
         expectThrows(ResponseException.class, () -> client().performRequest(new Request("GET", "/logs/_search")));
 
         // Add logs-* -> logs
@@ -155,8 +154,7 @@ public class DataStreamsRestIT extends ESRestTestCase {
 
         getAliasesRequest = new Request("GET", "/_aliases");
         getAliasesResponse = entityAsMap(client().performRequest(getAliasesRequest));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-myapp1.aliases", getAliasesResponse));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-myapp2.aliases", getAliasesResponse));
+        assertEquals(Map.of(), getAliasesResponse);
         expectThrows(ResponseException.class, () -> client().performRequest(new Request("GET", "/logs/_search")));
     }
 
@@ -225,20 +223,17 @@ public class DataStreamsRestIT extends ESRestTestCase {
         response = client().performRequest(new Request("GET", "/_alias/emea"));
         assertOK(response);
         getAliasesResponse = entityAsMap(response);
-        assertThat(getAliasesResponse.size(), equalTo(2)); // Adjust to equalTo(1) when #72953 is merged
+        assertThat(getAliasesResponse.size(), equalTo(1));
         assertEquals(Map.of("emea", Map.of()), XContentMapValues.extractValue("logs-emea.aliases", getAliasesResponse));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-nasa.aliases", getAliasesResponse)); // Remove when #72953 is merged
 
         ResponseException exception =
             expectThrows(ResponseException.class, () -> client().performRequest(new Request("GET", "/_alias/wrong_name")));
         response = exception.getResponse();
         assertThat(response.getStatusLine().getStatusCode(), equalTo(404));
         getAliasesResponse = entityAsMap(response);
-        assertThat(getAliasesResponse.size(), equalTo(4)); // Adjust to equalTo(2) when #72953 is merged
+        assertThat(getAliasesResponse.size(), equalTo(2));
         assertEquals("alias [wrong_name] missing", getAliasesResponse.get("error"));
         assertEquals(404, getAliasesResponse.get("status"));
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-emea.aliases", getAliasesResponse)); // Remove when #72953 is merged
-        assertEquals(Map.of(), XContentMapValues.extractValue("logs-nasa.aliases", getAliasesResponse)); // Remove when #72953 is merged
     }
 
 }