Browse Source

Add support for RestGetMapping cancellation (#72234)

Francisco Fernández Castaño 4 years ago
parent
commit
8997027f50

+ 120 - 0
qa/smoke-test-http/src/test/java/org/elasticsearch/http/RestGetMappingsCancellationIT.java

@@ -0,0 +1,120 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.http;
+
+import org.apache.http.client.methods.HttpGet;
+import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsAction;
+import org.elasticsearch.action.support.PlainActionFuture;
+import org.elasticsearch.action.support.master.AcknowledgedResponse;
+import org.elasticsearch.client.Cancellable;
+import org.elasticsearch.client.Request;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.ResponseListener;
+import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ack.AckedRequest;
+import org.elasticsearch.cluster.block.ClusterBlock;
+import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.test.ESIntegTestCase;
+
+import java.util.EnumSet;
+import java.util.concurrent.CancellationException;
+import java.util.function.Function;
+
+import static org.elasticsearch.test.TaskAssertions.assertAllCancellableTasksAreCancelled;
+import static org.elasticsearch.test.TaskAssertions.assertAllTasksHaveFinished;
+import static org.elasticsearch.test.TaskAssertions.awaitTaskWithPrefix;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0)
+public class RestGetMappingsCancellationIT extends HttpSmokeTestCase {
+
+    public void testGetMappingsCancellation() throws Exception {
+        internalCluster().startMasterOnlyNode();
+        internalCluster().startDataOnlyNode();
+        ensureStableCluster(2);
+
+        createIndex("test");
+        ensureGreen("test");
+        final String actionName = GetMappingsAction.NAME;
+        // Add a retryable cluster block that would block the request execution
+        updateClusterState(currentState -> {
+            ClusterBlock clusterBlock = new ClusterBlock(1000,
+                "Get mappings cancellation test cluster block",
+                true,
+                false,
+                false,
+                RestStatus.BAD_REQUEST,
+                EnumSet.of(ClusterBlockLevel.METADATA_READ)
+            );
+
+            return ClusterState.builder(currentState)
+                .blocks(ClusterBlocks.builder().addGlobalBlock(clusterBlock).build())
+                .build();
+        });
+
+        final Request request = new Request(HttpGet.METHOD_NAME, "/test/_mappings");
+        final PlainActionFuture<Void> future = new PlainActionFuture<>();
+        final Cancellable cancellable = getRestClient().performRequestAsync(request, new ResponseListener() {
+            @Override
+            public void onSuccess(Response response) {
+                future.onResponse(null);
+            }
+
+            @Override
+            public void onFailure(Exception exception) {
+                future.onFailure(exception);
+            }
+        });
+
+        assertThat(future.isDone(), equalTo(false));
+        awaitTaskWithPrefix(actionName);
+
+        cancellable.cancel();
+        assertAllCancellableTasksAreCancelled(actionName);
+
+        // Remove the cluster block
+        updateClusterState(currentState -> ClusterState.builder(currentState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build());
+
+        expectThrows(CancellationException.class, future::actionGet);
+
+        assertAllTasksHaveFinished(actionName);
+    }
+
+    private void updateClusterState(Function<ClusterState, ClusterState> transformationFn) {
+        final TimeValue timeout = TimeValue.timeValueSeconds(10);
+
+        final AckedRequest ackedRequest = new AckedRequest() {
+            @Override
+            public TimeValue ackTimeout() {
+                return timeout;
+            }
+
+            @Override
+            public TimeValue masterNodeTimeout() {
+                return timeout;
+            }
+        };
+
+        PlainActionFuture<AcknowledgedResponse> future = PlainActionFuture.newFuture();
+        internalCluster().getMasterNodeInstance(ClusterService.class).submitStateUpdateTask("get_mappings_cancellation_test",
+            new AckedClusterStateUpdateTask(ackedRequest, future) {
+                @Override
+                public ClusterState execute(ClusterState currentState) throws Exception {
+                    return transformationFn.apply(currentState);
+                }
+            });
+
+        future.actionGet();
+    }
+}

+ 5 - 2
server/src/main/java/org/elasticsearch/action/admin/indices/get/TransportGetIndexAction.java

@@ -17,6 +17,7 @@ import org.elasticsearch.cluster.metadata.AliasMetadata;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.metadata.MappingMetadata;
+import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.inject.Inject;
@@ -24,6 +25,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsFilter;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
@@ -53,7 +55,7 @@ public class TransportGetIndexAction extends TransportClusterInfoAction<GetIndex
     }
 
     @Override
-    protected void doMasterOperation(final GetIndexRequest request, String[] concreteIndices, final ClusterState state,
+    protected void doMasterOperation(Task task, final GetIndexRequest request, String[] concreteIndices, final ClusterState state,
                                      final ActionListener<GetIndexResponse> listener) {
         ImmutableOpenMap<String, MappingMetadata> mappingsResult = ImmutableOpenMap.of();
         ImmutableOpenMap<String, List<AliasMetadata>> aliasesResult = ImmutableOpenMap.of();
@@ -70,7 +72,8 @@ public class TransportGetIndexAction extends TransportClusterInfoAction<GetIndex
             switch (feature) {
             case MAPPINGS:
                     if (doneMappings == false) {
-                        mappingsResult = state.metadata().findMappings(concreteIndices, indicesService.getFieldFilter());
+                        mappingsResult = state.metadata()
+                            .findMappings(concreteIndices, indicesService.getFieldFilter(), Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
                         doneMappings = true;
                     }
                     break;

+ 8 - 0
server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsRequest.java

@@ -11,8 +11,12 @@ package org.elasticsearch.action.admin.indices.mapping.get;
 import org.elasticsearch.action.ActionRequestValidationException;
 import org.elasticsearch.action.support.master.info.ClusterInfoRequest;
 import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.tasks.CancellableTask;
+import org.elasticsearch.tasks.Task;
+import org.elasticsearch.tasks.TaskId;
 
 import java.io.IOException;
+import java.util.Map;
 
 public class GetMappingsRequest extends ClusterInfoRequest<GetMappingsRequest> {
 
@@ -28,4 +32,8 @@ public class GetMappingsRequest extends ClusterInfoRequest<GetMappingsRequest> {
         return null;
     }
 
+    @Override
+    public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        return new CancellableTask(id, type, action, "", parentTaskId, headers);
+    }
 }

+ 18 - 2
server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetMappingsAction.java

@@ -15,12 +15,19 @@ import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.info.TransportClusterInfoAction;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
+import org.elasticsearch.cluster.metadata.MappingMetadata;
+import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.tasks.CancellableTask;
+import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.concurrent.CancellationException;
+
 public class TransportGetMappingsAction extends TransportClusterInfoAction<GetMappingsRequest, GetMappingsResponse> {
 
     private static final Logger logger = LogManager.getLogger(TransportGetMappingsAction.class);
@@ -37,9 +44,18 @@ public class TransportGetMappingsAction extends TransportClusterInfoAction<GetMa
     }
 
     @Override
-    protected void doMasterOperation(final GetMappingsRequest request, String[] concreteIndices, final ClusterState state,
+    protected void doMasterOperation(Task task, final GetMappingsRequest request, String[] concreteIndices, final ClusterState state,
                                      final ActionListener<GetMappingsResponse> listener) {
         logger.trace("serving getMapping request based on version {}", state.version());
-        listener.onResponse(new GetMappingsResponse(state.metadata().findMappings(concreteIndices, indicesService.getFieldFilter())));
+        final Metadata metadata = state.metadata();
+        final ImmutableOpenMap<String, MappingMetadata> mappings =
+            metadata.findMappings(concreteIndices, indicesService.getFieldFilter(), () -> checkCancellation(task));
+        listener.onResponse(new GetMappingsResponse(mappings));
+    }
+
+    private void checkCancellation(Task task) {
+        if (task instanceof CancellableTask && ((CancellableTask) task).isCancelled()) {
+            throw new CancellationException("Task cancelled");
+        }
     }
 }

+ 3 - 3
server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java

@@ -29,7 +29,7 @@ public abstract class TransportClusterInfoAction<Request extends ClusterInfoRequ
                                       Writeable.Reader<Request> request, IndexNameExpressionResolver indexNameExpressionResolver,
                                       Writeable.Reader<Response> response) {
         super(actionName, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver, response,
-                ThreadPool.Names.SAME);
+            ThreadPool.Names.SAME);
     }
 
     @Override
@@ -42,9 +42,9 @@ public abstract class TransportClusterInfoAction<Request extends ClusterInfoRequ
     protected final void masterOperation(Task task, final Request request, final ClusterState state,
                                          final ActionListener<Response> listener) {
         String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request);
-        doMasterOperation(request, concreteIndices, state, listener);
+        doMasterOperation(task, request, concreteIndices, state, listener);
     }
 
-    protected abstract void doMasterOperation(Request request, String[] concreteIndices, ClusterState state,
+    protected abstract void doMasterOperation(Task task, Request request, String[] concreteIndices, ClusterState state,
                                               ActionListener<Response> listener);
 }

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

@@ -76,6 +76,7 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
 
     private static final Logger logger = LogManager.getLogger(Metadata.class);
 
+    public static final Runnable ON_NEXT_INDEX_FIND_MAPPINGS_NOOP = () -> { };
     public static final String ALL = "_all";
     public static final String UNKNOWN_CLUSTER_UUID = "_na_";
 
@@ -371,9 +372,11 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
      *
      * @see MapperPlugin#getFieldFilter()
      *
+     * @param onNextIndex a hook that gets notified for each index that's processed
      */
     public ImmutableOpenMap<String, MappingMetadata> findMappings(String[] concreteIndices,
-                                                                  Function<String, Predicate<String>> fieldFilter) {
+                                                                  Function<String, Predicate<String>> fieldFilter,
+                                                                  Runnable onNextIndex) {
         assert concreteIndices != null;
         if (concreteIndices.length == 0) {
             return ImmutableOpenMap.of();
@@ -382,6 +385,7 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
         ImmutableOpenMap.Builder<String, MappingMetadata> indexMapBuilder = ImmutableOpenMap.builder();
         Iterable<String> intersection = HppcMaps.intersection(ObjectHashSet.from(concreteIndices), indices.keys());
         for (String index : intersection) {
+            onNextIndex.run();
             IndexMetadata indexMetadata = indices.get(index);
             Predicate<String> fieldPredicate = fieldFilter.apply(index);
             indexMapBuilder.put(index, filterFields(indexMetadata.mapping(), fieldPredicate));

+ 35 - 28
server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.rest.action.admin.indices;
 
 import org.elasticsearch.ElasticsearchTimeoutException;
-import org.elasticsearch.action.ActionRunnable;
 import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
 import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
 import org.elasticsearch.action.support.IndicesOptions;
@@ -18,18 +17,18 @@ import org.elasticsearch.common.RestApiVersion;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.logging.DeprecationLogger;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.http.HttpChannel;
 import org.elasticsearch.rest.BaseRestHandler;
-import org.elasticsearch.rest.BytesRestResponse;
 import org.elasticsearch.rest.RestRequest;
-import org.elasticsearch.rest.RestResponse;
-import org.elasticsearch.rest.RestStatus;
-import org.elasticsearch.rest.action.RestActionListener;
-import org.elasticsearch.rest.action.RestBuilderListener;
+import org.elasticsearch.rest.action.DispatchingRestToXContentListener;
+import org.elasticsearch.rest.action.RestCancellableNodeClient;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.function.LongSupplier;
 
 import static org.elasticsearch.rest.RestRequest.Method.GET;
 
@@ -73,28 +72,36 @@ public class RestGetMappingAction extends BaseRestHandler {
         final TimeValue timeout = request.paramAsTime("master_timeout", getMappingsRequest.masterNodeTimeout());
         getMappingsRequest.masterNodeTimeout(timeout);
         getMappingsRequest.local(request.paramAsBoolean("local", getMappingsRequest.local()));
-        return channel -> client.admin().indices().getMappings(getMappingsRequest, new RestActionListener<>(channel) {
-
-            @Override
-            protected void processResponse(GetMappingsResponse getMappingsResponse) {
-                final long startTimeMs = threadPool.relativeTimeInMillis();
-                // Process serialization on MANAGEMENT pool since the serialization of the raw mappings to XContent can be too slow to
-                // execute on an IO thread
-                threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(
-                        ActionRunnable.wrap(this, l -> new RestBuilderListener<GetMappingsResponse>(channel) {
-                            @Override
-                            public RestResponse buildResponse(final GetMappingsResponse response,
-                                                              final XContentBuilder builder) throws Exception {
-                                if (threadPool.relativeTimeInMillis() - startTimeMs > timeout.millis()) {
-                                    throw new ElasticsearchTimeoutException("Timed out getting mappings");
-                                }
-                                builder.startObject();
-                                response.toXContent(builder, request);
-                                builder.endObject();
-                                return new BytesRestResponse(RestStatus.OK, builder);
-                            }
-                        }.onResponse(getMappingsResponse)));
+        final HttpChannel httpChannel = request.getHttpChannel();
+        return channel -> new RestCancellableNodeClient(client, httpChannel).admin().indices().getMappings(getMappingsRequest,
+                new DispatchingRestToXContentListener<>(threadPool.executor(ThreadPool.Names.MANAGEMENT), channel, request)
+                    .map(getMappingsResponse ->
+                        new RestGetMappingsResponse(getMappingsResponse, threadPool::relativeTimeInMillis, timeout)));
+    }
+
+    private static final class RestGetMappingsResponse implements ToXContentObject {
+        private final GetMappingsResponse response;
+        private final LongSupplier relativeTimeSupplierMillis;
+        private final TimeValue timeout;
+        private final long startTimeMs;
+
+        private RestGetMappingsResponse(GetMappingsResponse response, LongSupplier relativeTimeSupplierMillis, TimeValue timeout) {
+            this.response = response;
+            this.relativeTimeSupplierMillis = relativeTimeSupplierMillis;
+            this.timeout = timeout;
+            this.startTimeMs = relativeTimeSupplierMillis.getAsLong();
+        }
+
+        @Override
+        public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+            if (relativeTimeSupplierMillis.getAsLong() - startTimeMs > timeout.millis()) {
+                throw new ElasticsearchTimeoutException("Timed out getting mappings");
             }
-        });
+
+            builder.startObject();
+            response.toXContent(builder, params);
+            builder.endObject();
+            return builder;
+        }
     }
 }

+ 4 - 3
server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.indices.EmptySystemIndices;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.tasks.Task;
 import org.elasticsearch.test.ESSingleNodeTestCase;
 import org.elasticsearch.test.transport.CapturingTransport;
 import org.elasticsearch.threadpool.TestThreadPool;
@@ -105,10 +106,10 @@ public class GetIndexActionTests extends ESSingleNodeTestCase {
         }
 
         @Override
-        protected void doMasterOperation(GetIndexRequest request, String[] concreteIndices, ClusterState state,
-                                       ActionListener<GetIndexResponse> listener) {
+        protected void doMasterOperation(Task task, GetIndexRequest request, String[] concreteIndices, ClusterState state,
+                                         ActionListener<GetIndexResponse> listener) {
             ClusterState stateWithIndex = ClusterStateCreationUtils.state(indexName, 1, 1);
-            super.doMasterOperation(request, concreteIndices, stateWithIndex, listener);
+            super.doMasterOperation(task, request, concreteIndices, stateWithIndex, listener);
         }
     }
 

+ 23 - 8
server/src/test/java/org/elasticsearch/cluster/metadata/MetadataTests.java

@@ -42,6 +42,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
 import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createBackingIndex;
@@ -566,23 +567,32 @@ public class MetadataTests extends ESTestCase {
                     .putMapping(FIND_MAPPINGS_TEST_ITEM)).build();
 
         {
+            AtomicInteger onNextIndexCalls = new AtomicInteger(0);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(Strings.EMPTY_ARRAY,
-                    MapperPlugin.NOOP_FIELD_FILTER);
+                    MapperPlugin.NOOP_FIELD_FILTER,
+                    onNextIndexCalls::incrementAndGet);
             assertEquals(0, mappings.size());
+            assertThat(onNextIndexCalls.get(), equalTo(0));
         }
         {
+            AtomicInteger onNextIndexCalls = new AtomicInteger(0);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index1"},
-                    MapperPlugin.NOOP_FIELD_FILTER);
+                    MapperPlugin.NOOP_FIELD_FILTER,
+                    onNextIndexCalls::incrementAndGet);
             assertEquals(1, mappings.size());
             assertIndexMappingsNotFiltered(mappings, "index1");
+            assertThat(onNextIndexCalls.get(), equalTo(1));
         }
         {
+            AtomicInteger onNextIndexCalls = new AtomicInteger(0);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(
                     new String[]{"index1", "index2"},
-                    MapperPlugin.NOOP_FIELD_FILTER);
+                    MapperPlugin.NOOP_FIELD_FILTER,
+                onNextIndexCalls::incrementAndGet);
             assertEquals(2, mappings.size());
             assertIndexMappingsNotFiltered(mappings, "index1");
             assertIndexMappingsNotFiltered(mappings, "index2");
+            assertThat(onNextIndexCalls.get(), equalTo(2));
         }
     }
 
@@ -598,13 +608,15 @@ public class MetadataTests extends ESTestCase {
 
         {
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index1"},
-                    MapperPlugin.NOOP_FIELD_FILTER);
+                    MapperPlugin.NOOP_FIELD_FILTER,
+                    Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata mappingMetadata = mappings.get("index1");
             assertSame(originalMappingMetadata, mappingMetadata);
         }
         {
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index1"},
-                    index -> field -> randomBoolean());
+                    index -> field -> randomBoolean(),
+                    Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata mappingMetadata = mappings.get("index1");
             assertNotSame(originalMappingMetadata, mappingMetadata);
         }
@@ -648,7 +660,8 @@ public class MetadataTests extends ESTestCase {
                             return field -> false;
                         }
                         return MapperPlugin.NOOP_FIELD_PREDICATE;
-                    });
+                    },
+                    Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
 
 
 
@@ -699,7 +712,8 @@ public class MetadataTests extends ESTestCase {
         {
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(
                     new String[]{"index1", "index2" , "index3"},
-                    index -> field -> (index.equals("index3") && field.endsWith("keyword")));
+                    index -> field -> (index.equals("index3") && field.endsWith("keyword")),
+                    Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
 
             assertIndexMappingsNoFields(mappings, "index1");
             assertIndexMappingsNoFields(mappings, "index2");
@@ -733,7 +747,8 @@ public class MetadataTests extends ESTestCase {
         {
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(
                     new String[]{"index1", "index2" , "index3"},
-                    index -> field -> (index.equals("index2")));
+                    index -> field -> (index.equals("index2")),
+                    Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
 
             assertIndexMappingsNoFields(mappings, "index1");
             assertIndexMappingsNoFields(mappings, "index3");

+ 4 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/ElasticsearchMappings.java

@@ -19,6 +19,7 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexAbstraction;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.MappingMetadata;
+import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.common.CheckedSupplier;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.xcontent.XContentType;
@@ -102,7 +103,9 @@ public class ElasticsearchMappings {
         List<String> indicesToUpdate = new ArrayList<>();
 
         ImmutableOpenMap<String, MappingMetadata> currentMapping = state.metadata().findMappings(concreteIndices,
-                MapperPlugin.NOOP_FIELD_FILTER);
+                MapperPlugin.NOOP_FIELD_FILTER,
+                Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP
+        );
 
         for (String index : concreteIndices) {
             MappingMetadata metadata = currentMapping.get(index);

+ 4 - 4
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/FieldSubsetReaderTests.java

@@ -1029,7 +1029,7 @@ public class FieldSubsetReaderTests extends ESTestCase {
             FieldPermissionsDefinition definition = new FieldPermissionsDefinition(new String[]{"*inner1"}, Strings.EMPTY_ARRAY);
             FieldPermissions fieldPermissions = new FieldPermissions(definition);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index"},
-                    index -> fieldPermissions::grantsAccessTo);
+                    index -> fieldPermissions::grantsAccessTo, Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata index = mappings.get("index");
             Map<String, Object> sourceAsMap = index.getSourceAsMap();
             assertEquals(1, sourceAsMap.size());
@@ -1066,7 +1066,7 @@ public class FieldSubsetReaderTests extends ESTestCase {
             FieldPermissionsDefinition definition = new FieldPermissionsDefinition(new String[]{"object*"}, Strings.EMPTY_ARRAY);
             FieldPermissions fieldPermissions = new FieldPermissions(definition);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index"},
-                    index -> fieldPermissions::grantsAccessTo);
+                    index -> fieldPermissions::grantsAccessTo, Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata index = mappings.get("index");
             Map<String, Object> sourceAsMap = index.getSourceAsMap();
             assertEquals(1, sourceAsMap.size());
@@ -1102,7 +1102,7 @@ public class FieldSubsetReaderTests extends ESTestCase {
             FieldPermissionsDefinition definition = new FieldPermissionsDefinition(new String[]{"object"}, Strings.EMPTY_ARRAY);
             FieldPermissions fieldPermissions = new FieldPermissions(definition);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index"},
-                    index -> fieldPermissions::grantsAccessTo);
+                    index -> fieldPermissions::grantsAccessTo, Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata index = mappings.get("index");
             Map<String, Object> sourceAsMap = index.getSourceAsMap();
             assertEquals(1, sourceAsMap.size());
@@ -1128,7 +1128,7 @@ public class FieldSubsetReaderTests extends ESTestCase {
             FieldPermissionsDefinition definition = new FieldPermissionsDefinition(new String[]{"nested.inner2"}, Strings.EMPTY_ARRAY);
             FieldPermissions fieldPermissions = new FieldPermissions(definition);
             ImmutableOpenMap<String, MappingMetadata> mappings = metadata.findMappings(new String[]{"index"},
-                    index -> fieldPermissions::grantsAccessTo);
+                    index -> fieldPermissions::grantsAccessTo, Metadata.ON_NEXT_INDEX_FIND_MAPPINGS_NOOP);
             MappingMetadata index = mappings.get("index");
             Map<String, Object> sourceAsMap = index.getSourceAsMap();
             assertEquals(1, sourceAsMap.size());

+ 3 - 2
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportExplainLifecycleAction.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.xcontent.DeprecationHandler;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
+import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xpack.core.ilm.ErrorStep;
@@ -58,8 +59,8 @@ public class TransportExplainLifecycleAction
     }
 
     @Override
-    protected void doMasterOperation(ExplainLifecycleRequest request, String[] concreteIndices, ClusterState state,
-            ActionListener<ExplainLifecycleResponse> listener) {
+    protected void doMasterOperation(Task task, ExplainLifecycleRequest request, String[] concreteIndices, ClusterState state,
+                                     ActionListener<ExplainLifecycleResponse> listener) {
         Map<String, IndexLifecycleExplainResponse> indexResponses = new HashMap<>();
         for (String index : concreteIndices) {
             IndexMetadata idxMetadata = state.metadata().index(index);