diff --git a/docs/changelog/125652.yaml b/docs/changelog/125652.yaml new file mode 100644 index 0000000000000..59a69964a0967 --- /dev/null +++ b/docs/changelog/125652.yaml @@ -0,0 +1,5 @@ +pr: 125652 +summary: Run `TransportGetIndexAction` on local node +area: Indices APIs +type: enhancement +issues: [] diff --git a/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestActionCancellationIT.java b/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestActionCancellationIT.java index b4faacf4c0d51..c5ecf27e53296 100644 --- a/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestActionCancellationIT.java +++ b/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestActionCancellationIT.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterGetSettingsAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesAction; +import org.elasticsearch.action.admin.indices.get.GetIndexAction; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsAction; import org.elasticsearch.action.admin.indices.recovery.RecoveryAction; import org.elasticsearch.action.admin.indices.settings.get.GetSettingsAction; @@ -115,6 +116,11 @@ public void testGetMappingsCancellation() { runRestActionCancellationTest(new Request(HttpGet.METHOD_NAME, "/test/_mappings"), GetMappingsAction.NAME); } + public void testGetIndicesCancellation() { + createIndex("test"); + runRestActionCancellationTest(new Request(HttpGet.METHOD_NAME, "/test"), GetIndexAction.NAME); + } + public void testGetIndexSettingsCancellation() { createIndex("test"); runRestActionCancellationTest(new Request(HttpGet.METHOD_NAME, "/test/_settings"), GetSettingsAction.NAME); diff --git a/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestClusterInfoActionCancellationIT.java b/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestClusterInfoActionCancellationIT.java deleted file mode 100644 index 8f0e4ba82e283..0000000000000 --- a/qa/smoke-test-http/src/internalClusterTest/java/org/elasticsearch/http/RestClusterInfoActionCancellationIT.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", 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.get.GetIndexAction; -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.cluster.AckedClusterStateUpdateTask; -import org.elasticsearch.cluster.ClusterState; -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.core.TimeValue; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.test.ESIntegTestCase; -import org.hamcrest.Matchers; - -import java.util.EnumSet; -import java.util.concurrent.CancellationException; -import java.util.function.Function; - -import static org.elasticsearch.action.support.ActionTestUtils.wrapAsRestResponseListener; -import static org.elasticsearch.test.TaskAssertions.assertAllCancellableTasksAreCancelled; -import static org.elasticsearch.test.TaskAssertions.assertAllTasksHaveFinished; -import static org.elasticsearch.test.TaskAssertions.awaitTaskWithPrefixOnMaster; -import static org.hamcrest.core.IsEqual.equalTo; - -@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0) -public class RestClusterInfoActionCancellationIT extends HttpSmokeTestCase { - - public void testGetIndicesCancellation() throws Exception { - runTest(GetIndexAction.NAME, "/test"); - } - - private void runTest(String actionName, String endpoint) throws Exception { - internalCluster().startMasterOnlyNode(); - internalCluster().startDataOnlyNode(); - ensureStableCluster(2); - - createIndex("test"); - ensureGreen("test"); - // Add a retryable cluster block that would block the request execution - updateClusterState(currentState -> { - ClusterBlock clusterBlock = new ClusterBlock( - 1000, - actionName + " 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, endpoint); - final PlainActionFuture future = new PlainActionFuture<>(); - final Cancellable cancellable = getRestClient().performRequestAsync(request, wrapAsRestResponseListener(future)); - - assertThat(future.isDone(), equalTo(false)); - awaitTaskWithPrefixOnMaster(actionName); - // To ensure that the task is executing on master, we wait until the first blocked execution of the task registers its cluster state - // observer for further retries. This ensures that a task is not cancelled before we have started its execution, which could result - // in the task being unregistered and the test not being able to find any cancelled tasks. - assertBusy( - () -> assertThat( - internalCluster().getCurrentMasterNodeInstance(ClusterService.class) - .getClusterApplierService() - .getTimeoutClusterStateListenersSize(), - Matchers.greaterThan(0) - ) - ); - 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 transformationFn) { - final TimeValue timeout = TimeValue.timeValueSeconds(10); - PlainActionFuture future = new PlainActionFuture<>(); - internalCluster().getAnyMasterNodeInstance(ClusterService.class) - .submitUnbatchedStateUpdateTask("get_mappings_cancellation_test", new AckedClusterStateUpdateTask(timeout, timeout, future) { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - return transformationFn.apply(currentState); - } - }); - - future.actionGet(); - } -} diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.exists/10_basic.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.exists/10_basic.yml index e8bbb806d1d10..72e9e0c924dfc 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.exists/10_basic.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.exists/10_basic.yml @@ -17,9 +17,14 @@ - is_true: '' --- "Test indices.exists with local flag": + - requires: + test_runner_features: ["allowed_warnings"] + - do: indices.exists: index: test_index local: true + allowed_warnings: + - "the [?local] query parameter to this API has no effect, is now deprecated, and will be removed in a future version" - is_false: '' diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 053eb46fade2f..d135b67ac7401 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -144,7 +144,6 @@ exports org.elasticsearch.action.support.broadcast.node; exports org.elasticsearch.action.support.broadcast.unpromotable; exports org.elasticsearch.action.support.master; - exports org.elasticsearch.action.support.master.info; exports org.elasticsearch.action.support.nodes; exports org.elasticsearch.action.support.local; exports org.elasticsearch.action.support.replication; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java index 05c44b55cf8bc..84c040df758d3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java @@ -9,13 +9,16 @@ package org.elasticsearch.action.admin.indices.get; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.info.ClusterInfoRequest; +import org.elasticsearch.action.support.local.LocalClusterStateRequest; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.ArrayUtils; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.UpdateForV10; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; @@ -32,7 +35,7 @@ /** * A request to retrieve information about an index. */ -public class GetIndexRequest extends ClusterInfoRequest { +public class GetIndexRequest extends LocalClusterStateRequest implements IndicesRequest.Replaceable { public enum Feature { ALIASES((byte) 0), MAPPINGS((byte) 1), @@ -90,16 +93,30 @@ public static Feature[] fromRequest(RestRequest request) { } static final Feature[] DEFAULT_FEATURES = new Feature[] { Feature.ALIASES, Feature.MAPPINGS, Feature.SETTINGS }; + + private String[] indices = Strings.EMPTY_ARRAY; + private IndicesOptions indicesOptions; private Feature[] features = DEFAULT_FEATURES; private boolean humanReadable = false; private transient boolean includeDefaults = false; public GetIndexRequest(TimeValue masterTimeout) { - super(masterTimeout, IndicesOptions.strictExpandOpen()); + super(masterTimeout); + indicesOptions = IndicesOptions.strictExpandOpen(); } + /** + * NB prior to 9.1 this was a TransportMasterNodeReadAction so for BwC we must remain able to read these requests until + * we no longer need to support calling this action remotely. + */ + @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) public GetIndexRequest(StreamInput in) throws IOException { super(in); + indices = in.readStringArray(); + if (in.getTransportVersion().before(TransportVersions.V_8_0_0)) { + in.readStringArray(); + } + indicesOptions = IndicesOptions.readIndicesOptions(in); features = in.readArray(i -> Feature.fromId(i.readByte()), Feature[]::new); humanReadable = in.readBoolean(); includeDefaults = in.readBoolean(); @@ -162,11 +179,29 @@ public boolean includeDefaults() { } @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeArray((o, f) -> o.writeByte(f.id), features); - out.writeBoolean(humanReadable); - out.writeBoolean(includeDefaults); + public GetIndexRequest indices(String... indices) { + this.indices = indices; + return this; + } + + public GetIndexRequest indicesOptions(IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + return this; + } + + @Override + public String[] indices() { + return indices; + } + + @Override + public IndicesOptions indicesOptions() { + return indicesOptions; + } + + @Override + public boolean includeDataStreams() { + return true; } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequestBuilder.java index 18abb9e5c58e1..8ed708c616f76 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequestBuilder.java @@ -9,17 +9,34 @@ package org.elasticsearch.action.admin.indices.get; +import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.admin.indices.get.GetIndexRequest.Feature; -import org.elasticsearch.action.support.master.info.ClusterInfoRequestBuilder; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.ElasticsearchClient; +import org.elasticsearch.common.util.ArrayUtils; import org.elasticsearch.core.TimeValue; -public class GetIndexRequestBuilder extends ClusterInfoRequestBuilder { +public class GetIndexRequestBuilder extends ActionRequestBuilder { public GetIndexRequestBuilder(ElasticsearchClient client, TimeValue masterTimeout, String... indices) { super(client, GetIndexAction.INSTANCE, new GetIndexRequest(masterTimeout).indices(indices)); } + public GetIndexRequestBuilder setIndices(String... indices) { + request.indices(indices); + return this; + } + + public GetIndexRequestBuilder addIndices(String... indices) { + request.indices(ArrayUtils.concat(request.indices(), indices)); + return this; + } + + public GetIndexRequestBuilder setIndicesOptions(IndicesOptions indicesOptions) { + request.indicesOptions(indicesOptions); + return this; + } + public GetIndexRequestBuilder setFeatures(Feature... features) { request.features(features); return this; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java index 8e841219c24de..91f7dfd8a98f5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ChunkedToXContentObject; +import org.elasticsearch.core.UpdateForV10; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.xcontent.ToXContent; @@ -69,6 +70,11 @@ public GetIndexResponse( } } + /** + * The only usage of this constructor is for BwC cross-cluster transforms for clusters before v8.2. The ML team is aware that we + * don't need to support that anymore now that we're on v9. Once they remove that BwC code, we can remove this constructor as well. + */ + @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) GetIndexResponse(StreamInput in) throws IOException { this.indices = in.readStringArray(); mappings = in.readImmutableOpenMap(StreamInput::readString, in.getTransportVersion().before(TransportVersions.V_8_0_0) ? i -> { @@ -165,6 +171,11 @@ public String getSetting(String index, String setting) { } } + /** + * NB prior to 9.1 this was a TransportMasterNodeReadAction so for BwC we must remain able to write these responses until + * we no longer need to support calling this action remotely. + */ + @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) @Override public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(indices); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/TransportGetIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/TransportGetIndexAction.java index c0e1f8197ee85..bdd0c8b81eb4b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/TransportGetIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/TransportGetIndexAction.java @@ -12,8 +12,11 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.get.GetIndexRequest.Feature; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.master.info.TransportClusterInfoAction; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.local.TransportLocalProjectMetadataAction; +import org.elasticsearch.cluster.ProjectState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -25,6 +28,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.core.UpdateForV10; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.injection.guice.Inject; import org.elasticsearch.tasks.CancellableTask; @@ -41,12 +45,19 @@ /** * Get index action. */ -public class TransportGetIndexAction extends TransportClusterInfoAction { +public class TransportGetIndexAction extends TransportLocalProjectMetadataAction { private final IndicesService indicesService; private final IndexScopedSettings indexScopedSettings; private final SettingsFilter settingsFilter; + private final IndexNameExpressionResolver indexNameExpressionResolver; + /** + * NB prior to 9.1 this was a TransportMasterNodeReadAction so for BwC it must be registered with the TransportService until + * we no longer need to support calling this action remotely. + */ + @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) + @SuppressWarnings("this-escape") @Inject public TransportGetIndexAction( TransportService transportService, @@ -61,33 +72,50 @@ public TransportGetIndexAction( ) { super( GetIndexAction.NAME, - transportService, - clusterService, - threadPool, actionFilters, - GetIndexRequest::new, - indexNameExpressionResolver, - GetIndexResponse::new, + transportService.getTaskManager(), + clusterService, + threadPool.executor(ThreadPool.Names.MANAGEMENT), projectResolver ); this.indicesService = indicesService; this.settingsFilter = settingsFilter; this.indexScopedSettings = indexScopedSettings; + this.indexNameExpressionResolver = indexNameExpressionResolver; + + transportService.registerRequestHandler( + actionName, + executor, + false, + true, + GetIndexRequest::new, + (request, channel, task) -> executeDirect(task, request, new ChannelActionListener<>(channel)) + ); } @Override - protected void doMasterOperation( + protected ClusterBlockException checkBlock(GetIndexRequest request, ProjectState state) { + return state.blocks() + .indicesBlockedException( + state.projectId(), + ClusterBlockLevel.METADATA_READ, + indexNameExpressionResolver.concreteIndexNames(state.metadata(), request) + ); + } + + @Override + protected void localClusterStateOperation( Task task, final GetIndexRequest request, - String[] concreteIndices, - final ClusterState state, + final ProjectState state, final ActionListener listener - ) { + ) throws Exception { + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state.metadata(), request); Map mappingsResult = ImmutableOpenMap.of(); Map> aliasesResult = Map.of(); Map settings = Map.of(); Map defaultSettings = Map.of(); - ProjectMetadata project = projectResolver.getProjectMetadata(state); + ProjectMetadata project = state.metadata(); Map dataStreams = project.findDataStreams(concreteIndices) .entrySet() .stream() diff --git a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequest.java b/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequest.java deleted file mode 100644 index 2f3e48cbca0d9..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequest.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.support.master.info; - -import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.MasterNodeReadRequest; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.core.TimeValue; - -import java.io.IOException; - -public abstract class ClusterInfoRequest> extends MasterNodeReadRequest - implements - IndicesRequest.Replaceable { - - private String[] indices = Strings.EMPTY_ARRAY; - - private IndicesOptions indicesOptions; - - protected ClusterInfoRequest(TimeValue masterTimeout, IndicesOptions indicesOptions) { - super(masterTimeout); - this.indicesOptions = indicesOptions; - } - - protected ClusterInfoRequest(StreamInput in) throws IOException { - super(in); - indices = in.readStringArray(); - if (in.getTransportVersion().before(TransportVersions.V_8_0_0)) { - in.readStringArray(); - } - indicesOptions = IndicesOptions.readIndicesOptions(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArray(indices); - if (out.getTransportVersion().before(TransportVersions.V_8_0_0)) { - out.writeStringArray(Strings.EMPTY_ARRAY); - } - indicesOptions.writeIndicesOptions(out); - } - - @Override - @SuppressWarnings("unchecked") - public Request indices(String... indices) { - this.indices = indices; - return (Request) this; - } - - @SuppressWarnings("unchecked") - public Request indicesOptions(IndicesOptions indicesOptions) { - this.indicesOptions = indicesOptions; - return (Request) this; - } - - @Override - public String[] indices() { - return indices; - } - - @Override - public IndicesOptions indicesOptions() { - return indicesOptions; - } - - @Override - public boolean includeDataStreams() { - return true; - } -} diff --git a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java deleted file mode 100644 index 94db1c1dd6977..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/support/master/info/ClusterInfoRequestBuilder.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ -package org.elasticsearch.action.support.master.info; - -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.ActionType; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; -import org.elasticsearch.client.internal.ElasticsearchClient; -import org.elasticsearch.common.util.ArrayUtils; - -public abstract class ClusterInfoRequestBuilder< - Request extends ClusterInfoRequest, - Response extends ActionResponse, - Builder extends ClusterInfoRequestBuilder> extends MasterNodeReadOperationRequestBuilder< - Request, - Response, - Builder> { - - protected ClusterInfoRequestBuilder(ElasticsearchClient client, ActionType action, Request request) { - super(client, action, request); - } - - @SuppressWarnings("unchecked") - public Builder setIndices(String... indices) { - request.indices(indices); - return (Builder) this; - } - - @SuppressWarnings("unchecked") - public Builder addIndices(String... indices) { - request.indices(ArrayUtils.concat(request.indices(), indices)); - return (Builder) this; - } - - @SuppressWarnings("unchecked") - public Builder setIndicesOptions(IndicesOptions indicesOptions) { - request.indicesOptions(indicesOptions); - return (Builder) this; - } -} diff --git a/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java b/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java deleted file mode 100644 index ecbd692fd19a1..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/support/master/info/TransportClusterInfoAction.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ -package org.elasticsearch.action.support.master.info; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.ProjectId; -import org.elasticsearch.cluster.metadata.ProjectMetadata; -import org.elasticsearch.cluster.project.ProjectResolver; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -public abstract class TransportClusterInfoAction, Response extends ActionResponse> extends - TransportMasterNodeReadAction { - - protected final ProjectResolver projectResolver; - private final IndexNameExpressionResolver indexNameExpressionResolver; - - public TransportClusterInfoAction( - String actionName, - TransportService transportService, - ClusterService clusterService, - ThreadPool threadPool, - ActionFilters actionFilters, - Writeable.Reader request, - IndexNameExpressionResolver indexNameExpressionResolver, - Writeable.Reader response, - ProjectResolver projectResolver - ) { - super( - actionName, - transportService, - clusterService, - threadPool, - actionFilters, - request, - response, - threadPool.executor(ThreadPool.Names.MANAGEMENT) - ); - this.projectResolver = projectResolver; - this.indexNameExpressionResolver = indexNameExpressionResolver; - } - - @Override - protected ClusterBlockException checkBlock(Request request, ClusterState state) { - final ProjectMetadata projectMetadata = projectResolver.getProjectMetadata(state); - return state.blocks() - .indicesBlockedException( - projectMetadata.id(), - ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(projectMetadata, request) - ); - } - - @Override - protected final void masterOperation( - Task task, - final Request request, - final ClusterState state, - final ActionListener listener - ) { - ProjectId projectId = projectResolver.getProjectId(); - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state.metadata().getProject(projectId), request); - doMasterOperation(task, request, concreteIndices, state, listener); - } - - protected abstract void doMasterOperation( - Task task, - Request request, - String[] concreteIndices, - ClusterState state, - ActionListener listener - ); -} diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java index e49efc80250ac..c7d0a1d713806 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestUtils; import org.elasticsearch.rest.Scope; import org.elasticsearch.rest.ServerlessScope; import org.elasticsearch.rest.action.RestCancellableNodeClient; @@ -27,7 +28,6 @@ import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.HEAD; -import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout; /** * The REST handler for get index and head index APIs. @@ -48,10 +48,10 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { String[] indices = Strings.splitStringByCommaToArray(request.param("index")); - final GetIndexRequest getIndexRequest = new GetIndexRequest(getMasterNodeTimeout(request)); + final GetIndexRequest getIndexRequest = new GetIndexRequest(RestUtils.getMasterNodeTimeout(request)); getIndexRequest.indices(indices); getIndexRequest.indicesOptions(IndicesOptions.fromRequest(request, getIndexRequest.indicesOptions())); - getIndexRequest.local(request.paramAsBoolean("local", getIndexRequest.local())); + RestUtils.consumeDeprecatedLocalParameter(request); getIndexRequest.humanReadable(request.paramAsBoolean("human", false)); getIndexRequest.includeDefaults(request.paramAsBoolean("include_defaults", false)); getIndexRequest.features(GetIndexRequest.Feature.fromRequest(request)); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java index 8e926830d04d6..765f7a85359da 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java @@ -14,10 +14,10 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.replication.ClusterStateCreationUtils; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ProjectState; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; -import org.elasticsearch.cluster.project.DefaultProjectResolver; import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -124,20 +124,19 @@ class TestTransportGetIndexAction extends TransportGetIndexAction { new GetIndexActionTests.Resolver(), indicesService, IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, - DefaultProjectResolver.INSTANCE + TestProjectResolvers.DEFAULT_PROJECT_ONLY ); } @Override - protected void doMasterOperation( + protected void localClusterStateOperation( Task task, GetIndexRequest request, - String[] concreteIndices, - ClusterState state, + ProjectState state, ActionListener listener - ) { - ClusterState stateWithIndex = ClusterStateCreationUtils.state(indexName, 1, 1); - super.doMasterOperation(task, request, concreteIndices, stateWithIndex, listener); + ) throws Exception { + ProjectState stateWithIndex = ClusterStateCreationUtils.state(indexName, 1, 1).projectState(ProjectId.DEFAULT); + super.localClusterStateOperation(task, request, stateWithIndex, listener); } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java index 9ca2acc0ed893..5edd1675b4e84 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java @@ -14,12 +14,11 @@ import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponseTests; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.RandomCreateIndexGenerator; import org.elasticsearch.test.AbstractChunkedSerializingTestCase; -import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; import java.io.IOException; import java.util.ArrayList; @@ -30,14 +29,8 @@ import java.util.Locale; import java.util.Map; -public class GetIndexResponseTests extends AbstractWireSerializingTestCase { +public class GetIndexResponseTests extends ESTestCase { - @Override - protected Writeable.Reader instanceReader() { - return GetIndexResponse::new; - } - - @Override protected GetIndexResponse createTestInstance() { String[] indices = generateRandomStringArray(5, 5, false, false); Map mappings = new HashMap<>(); @@ -77,11 +70,6 @@ protected GetIndexResponse createTestInstance() { return new GetIndexResponse(indices, mappings, aliases, settings, defaultSettings, dataStreams); } - @Override - protected GetIndexResponse mutateInstance(GetIndexResponse instance) { - return null;// TODO implement https://github.com/elastic/elasticsearch/issues/25929 - } - public void testChunking() throws IOException { AbstractChunkedSerializingTestCase.assertChunkCount(createTestInstance(), response -> response.getIndices().length + 2); } diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java index e910945830836..1136dde863749 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java @@ -263,8 +263,7 @@ private void resolveIndices( ) { if (retrieveIndices || retrieveFrozenIndices) { if (clusterIsLocal(clusterWildcard)) { // resolve local indices - GetIndexRequest indexRequest = new GetIndexRequest(MasterNodeRequest.INFINITE_MASTER_NODE_TIMEOUT).local(true) - .indices(indexWildcards) + GetIndexRequest indexRequest = new GetIndexRequest(MasterNodeRequest.INFINITE_MASTER_NODE_TIMEOUT).indices(indexWildcards) .features(Feature.SETTINGS) .includeDefaults(false) .indicesOptions(INDICES_ONLY_OPTIONS);