Run `TransportGetSettingsAction` on local node (#126051)

This action solely needs the cluster state, it can run on any node.
Additionally, it needs to be cancellable to avoid doing unnecessary work
after a client failure or timeout.

Relates #101805
This commit is contained in:
Niels Bauman 2025-04-02 15:05:31 +01:00 committed by GitHub
parent ea8fd3b77c
commit eb4d64f94a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 90 additions and 173 deletions

View File

@ -0,0 +1,5 @@
pr: 126051
summary: Run `TransportGetSettingsAction` on local node
area: Indices APIs
type: enhancement
issues: []

View File

@ -17,6 +17,7 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.action.admin.indices.alias.get.GetAliasesAction;
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;
import org.elasticsearch.action.admin.indices.template.get.GetComponentTemplateAction;
import org.elasticsearch.action.admin.indices.template.get.GetComposableIndexTemplateAction;
import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesAction;
@ -114,6 +115,11 @@ public class RestActionCancellationIT extends HttpSmokeTestCase {
runRestActionCancellationTest(new Request(HttpGet.METHOD_NAME, "/test/_mappings"), GetMappingsAction.NAME);
}
public void testGetIndexSettingsCancellation() {
createIndex("test");
runRestActionCancellationTest(new Request(HttpGet.METHOD_NAME, "/test/_settings"), GetSettingsAction.NAME);
}
private void runRestActionCancellationTest(Request request, String actionName) {
final var node = usually() ? internalCluster().getRandomNodeName() : internalCluster().startCoordinatingOnlyNode(Settings.EMPTY);

View File

@ -89,6 +89,7 @@
"description":"Return settings in flat format (default: false)"
},
"local":{
"deprecated":true,
"type":"boolean",
"description":"Return local information, do not retrieve the state from master node (default: false)"
},

View File

@ -164,10 +164,14 @@ setup:
---
"Get /_settings with local flag":
- requires:
test_runner_features: ["allowed_warnings"]
- do:
indices.get_settings:
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_true: test_1
- is_true: test_2

View File

@ -34,8 +34,6 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.TransportShardRefreshAction;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsAction;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsRequest;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsAction;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest;
import org.elasticsearch.action.admin.indices.settings.put.TransportUpdateSettingsAction;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction;
@ -524,16 +522,6 @@ public class IndicesRequestIT extends ESIntegTestCase {
assertSameIndices(putMappingRequest, TransportPutMappingAction.TYPE.name());
}
public void testGetSettings() {
interceptTransportActions(GetSettingsAction.NAME);
GetSettingsRequest getSettingsRequest = new GetSettingsRequest(TEST_REQUEST_TIMEOUT).indices(randomIndicesOrAliases());
internalCluster().coordOnlyNodeClient().admin().indices().getSettings(getSettingsRequest).actionGet();
clearInterceptedActions();
assertSameIndices(getSettingsRequest, GetSettingsAction.NAME);
}
public void testUpdateSettings() {
interceptTransportActions(TransportUpdateSettingsAction.TYPE.name());

View File

@ -13,17 +13,21 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.ValidateActions;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
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.core.TimeValue;
import org.elasticsearch.core.UpdateForV10;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import java.util.Objects;
public class GetSettingsRequest extends MasterNodeReadRequest<GetSettingsRequest> implements IndicesRequest.Replaceable {
public class GetSettingsRequest extends LocalClusterStateRequest implements IndicesRequest.Replaceable {
public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.fromOptions(false, true, true, true);
@ -57,6 +61,11 @@ public class GetSettingsRequest extends MasterNodeReadRequest<GetSettingsRequest
super(masterTimeout);
}
/**
* 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 GetSettingsRequest(StreamInput in) throws IOException {
super(in);
indices = in.readStringArray();
@ -66,16 +75,6 @@ public class GetSettingsRequest extends MasterNodeReadRequest<GetSettingsRequest
includeDefaults = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringArray(indices);
indicesOptions.writeIndicesOptions(out);
out.writeStringArray(names);
out.writeBoolean(humanReadable);
out.writeBoolean(includeDefaults);
}
@Override
public String[] indices() {
return indices;
@ -122,6 +121,11 @@ public class GetSettingsRequest extends MasterNodeReadRequest<GetSettingsRequest
return validationException;
}
@Override
public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
return new CancellableTask(id, type, action, "", parentTaskId, headers);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;

View File

@ -9,16 +9,13 @@
package org.elasticsearch.action.admin.indices.settings.get;
import org.elasticsearch.action.ActionRequestBuilder;
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;
import org.elasticsearch.core.TimeValue;
public class GetSettingsRequestBuilder extends MasterNodeReadOperationRequestBuilder<
GetSettingsRequest,
GetSettingsResponse,
GetSettingsRequestBuilder> {
public class GetSettingsRequestBuilder extends ActionRequestBuilder<GetSettingsRequest, GetSettingsResponse> {
public GetSettingsRequestBuilder(ElasticsearchClient client, TimeValue masterTimeout, String... indices) {
super(client, GetSettingsAction.INSTANCE, new GetSettingsRequest(masterTimeout).indices(indices));

View File

@ -12,10 +12,10 @@ package org.elasticsearch.action.admin.indices.settings.get;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.io.stream.StreamInput;
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.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.json.JsonXContent;
@ -36,11 +36,6 @@ public class GetSettingsResponse extends ActionResponse implements ChunkedToXCon
this.indexToDefaultSettings = indexToDefaultSettings;
}
public GetSettingsResponse(StreamInput in) throws IOException {
indexToSettings = in.readImmutableMap(Settings::readSettingsFromStream);
indexToDefaultSettings = in.readImmutableMap(Settings::readSettingsFromStream);
}
/**
* Returns a map of index name to {@link Settings} object. The returned {@link Settings}
* objects contain only those settings explicitly set on a given index.
@ -82,6 +77,11 @@ public class GetSettingsResponse extends ActionResponse implements ChunkedToXCon
}
}
/**
* 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.writeMap(indexToSettings, StreamOutput::writeWriteable);

View File

@ -11,12 +11,14 @@ package org.elasticsearch.action.admin.indices.settings.get;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
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.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.project.ProjectResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.IndexScopedSettings;
@ -24,6 +26,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.util.Maps;
import org.elasticsearch.core.UpdateForV10;
import org.elasticsearch.index.Index;
import org.elasticsearch.injection.guice.Inject;
import org.elasticsearch.tasks.Task;
@ -35,12 +38,18 @@ import java.util.Map;
import static java.util.Collections.unmodifiableMap;
public class TransportGetSettingsAction extends TransportMasterNodeReadAction<GetSettingsRequest, GetSettingsResponse> {
public class TransportGetSettingsAction extends TransportLocalProjectMetadataAction<GetSettingsRequest, GetSettingsResponse> {
private final SettingsFilter settingsFilter;
private final IndexScopedSettings indexScopedSettings;
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 TransportGetSettingsAction(
TransportService transportService,
@ -48,28 +57,40 @@ public class TransportGetSettingsAction extends TransportMasterNodeReadAction<Ge
ThreadPool threadPool,
SettingsFilter settingsFilter,
ActionFilters actionFilters,
ProjectResolver projectResolver,
IndexNameExpressionResolver indexNameExpressionResolver,
IndexScopedSettings indexedScopedSettings
) {
super(
GetSettingsAction.NAME,
transportService,
clusterService,
threadPool,
actionFilters,
GetSettingsRequest::new,
GetSettingsResponse::new,
threadPool.executor(ThreadPool.Names.MANAGEMENT)
transportService.getTaskManager(),
clusterService,
threadPool.executor(ThreadPool.Names.MANAGEMENT),
projectResolver
);
this.settingsFilter = settingsFilter;
this.indexScopedSettings = indexedScopedSettings;
this.indexNameExpressionResolver = indexNameExpressionResolver;
transportService.registerRequestHandler(
actionName,
executor,
false,
true,
GetSettingsRequest::new,
(request, channel, task) -> executeDirect(task, request, new ChannelActionListener<>(channel))
);
}
@Override
protected ClusterBlockException checkBlock(GetSettingsRequest request, ClusterState state) {
protected ClusterBlockException checkBlock(GetSettingsRequest request, ProjectState state) {
return state.blocks()
.indicesBlockedException(ClusterBlockLevel.METADATA_READ, indexNameExpressionResolver.concreteIndexNames(state, request));
.indicesBlockedException(
state.projectId(),
ClusterBlockLevel.METADATA_READ,
indexNameExpressionResolver.concreteIndexNames(state.metadata(), request)
);
}
private static boolean isFilteredRequest(GetSettingsRequest request) {
@ -77,20 +98,20 @@ public class TransportGetSettingsAction extends TransportMasterNodeReadAction<Ge
}
@Override
protected void masterOperation(
protected void localClusterStateOperation(
Task task,
GetSettingsRequest request,
ClusterState state,
ProjectState state,
ActionListener<GetSettingsResponse> listener
) {
assert Transports.assertNotTransportThread("O(indices) work is too much for a transport thread");
final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request);
final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state.metadata(), request);
final Map<String, Settings> indexToSettings = Maps.newHashMapWithExpectedSize(concreteIndices.length);
final Map<String, Settings> indexToDefaultSettings = request.includeDefaults()
? Maps.newHashMapWithExpectedSize(concreteIndices.length)
: null;
for (Index concreteIndex : concreteIndices) {
IndexMetadata indexMetadata = state.getMetadata().findIndex(concreteIndex).orElse(null);
IndexMetadata indexMetadata = state.metadata().index(concreteIndex);
if (indexMetadata == null) {
continue;
}

View File

@ -19,6 +19,7 @@ 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;
import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener;
import java.io.IOException;
@ -58,7 +59,9 @@ public class RestGetSettingsAction extends BaseRestHandler {
.humanReadable(request.hasParam("human"))
.includeDefaults(renderDefaults)
.names(names);
getSettingsRequest.local(request.paramAsBoolean("local", getSettingsRequest.local()));
return channel -> client.admin().indices().getSettings(getSettingsRequest, new RestRefCountedChunkedToXContentListener<>(channel));
RestUtils.consumeDeprecatedLocalParameter(request);
return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).admin()
.indices()
.getSettings(getSettingsRequest, new RestRefCountedChunkedToXContentListener<>(channel));
}
}

View File

@ -123,7 +123,6 @@ public class RestIndicesAction extends AbstractCatAction {
.indices()
.prepareGetSettings(masterNodeTimeout, indices)
.setIndicesOptions(indicesOptions)
.setMasterNodeTimeout(masterNodeTimeout)
.execute(listeners.acquire(indexSettingsRef::set));
// The other requests just provide additional detail, and wildcards may be resolved differently depending on the type of

View File

@ -14,7 +14,7 @@ import org.elasticsearch.action.IndicesRequest;
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.ProjectMetadata;
import org.elasticsearch.cluster.project.TestProjectResolvers;
@ -59,20 +59,21 @@ public class GetSettingsActionTests extends ESTestCase {
GetSettingsActionTests.this.threadPool,
settingsFilter,
new ActionFilters(Collections.emptySet()),
TestProjectResolvers.DEFAULT_PROJECT_ONLY,
new Resolver(),
IndexScopedSettings.DEFAULT_SCOPED_SETTINGS
);
}
@Override
protected void masterOperation(
protected void localClusterStateOperation(
Task task,
GetSettingsRequest request,
ClusterState state,
ProjectState state,
ActionListener<GetSettingsResponse> listener
) {
ClusterState stateWithIndex = ClusterStateCreationUtils.state(indexName, 1, 1);
super.masterOperation(task, request, stateWithIndex, listener);
ProjectState stateWithIndex = ClusterStateCreationUtils.state(indexName, 1, 1).projectState(state.projectId());
super.localClusterStateOperation(task, request, stateWithIndex, listener);
}
}

View File

@ -1,34 +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.admin.indices.settings.get;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class GetSettingsRequestTests extends ESTestCase {
private static final GetSettingsRequest TEST_700_REQUEST = new GetSettingsRequest(TEST_REQUEST_TIMEOUT).includeDefaults(true)
.humanReadable(true)
.indices("test_index")
.names("test_setting_key");
public void testSerdeRoundTrip() throws IOException {
BytesStreamOutput bso = new BytesStreamOutput();
TEST_700_REQUEST.writeTo(bso);
byte[] responseBytes = BytesReference.toBytes(bso.bytes());
StreamInput si = StreamInput.wrap(responseBytes);
GetSettingsRequest deserialized = new GetSettingsRequest(si);
assertEquals(TEST_700_REQUEST, deserialized);
}
}

View File

@ -9,24 +9,18 @@
package org.elasticsearch.action.admin.indices.settings.get;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParserUtils;
import org.elasticsearch.index.RandomCreateIndexGenerator;
import org.elasticsearch.test.AbstractChunkedSerializingTestCase;
import org.elasticsearch.xcontent.XContentParser;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.function.Predicate;
public class GetSettingsResponseTests extends AbstractChunkedSerializingTestCase<GetSettingsResponse> {
public class GetSettingsResponseTests extends ESTestCase {
@Override
protected GetSettingsResponse createTestInstance() {
HashMap<String, Settings> indexToSettings = new HashMap<>();
HashMap<String, Settings> indexToDefaultSettings = new HashMap<>();
@ -61,78 +55,6 @@ public class GetSettingsResponseTests extends AbstractChunkedSerializingTestCase
return new GetSettingsResponse(indexToSettings, indexToDefaultSettings);
}
@Override
protected GetSettingsResponse mutateInstance(GetSettingsResponse instance) {
return null;// TODO implement https://github.com/elastic/elasticsearch/issues/25929
}
@Override
protected Writeable.Reader<GetSettingsResponse> instanceReader() {
return GetSettingsResponse::new;
}
@Override
protected GetSettingsResponse doParseInstance(XContentParser parser) throws IOException {
HashMap<String, Settings> indexToSettings = new HashMap<>();
HashMap<String, Settings> indexToDefaultSettings = new HashMap<>();
if (parser.currentToken() == null) {
parser.nextToken();
}
XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser);
parser.nextToken();
while (parser.isClosed() == false) {
if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
// we must assume this is an index entry
parseIndexEntry(parser, indexToSettings, indexToDefaultSettings);
} else if (parser.currentToken() == XContentParser.Token.START_ARRAY) {
parser.skipChildren();
} else {
parser.nextToken();
}
}
return new GetSettingsResponse(Map.copyOf(indexToSettings), Map.copyOf(indexToDefaultSettings));
}
private static void parseIndexEntry(
XContentParser parser,
Map<String, Settings> indexToSettings,
Map<String, Settings> indexToDefaultSettings
) throws IOException {
String indexName = parser.currentName();
parser.nextToken();
while (parser.isClosed() == false && parser.currentToken() != XContentParser.Token.END_OBJECT) {
parseSettingsField(parser, indexName, indexToSettings, indexToDefaultSettings);
}
}
private static void parseSettingsField(
XContentParser parser,
String currentIndexName,
Map<String, Settings> indexToSettings,
Map<String, Settings> indexToDefaultSettings
) throws IOException {
if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
switch (parser.currentName()) {
case "settings" -> indexToSettings.put(currentIndexName, Settings.fromXContent(parser));
case "defaults" -> indexToDefaultSettings.put(currentIndexName, Settings.fromXContent(parser));
default -> parser.skipChildren();
}
} else if (parser.currentToken() == XContentParser.Token.START_ARRAY) {
parser.skipChildren();
}
parser.nextToken();
}
@Override
protected Predicate<String> getRandomFieldsExcludeFilter() {
// we do not want to add new fields at the root (index-level), or inside settings blocks
return f -> f.equals("") || f.contains(".settings") || f.contains(".defaults");
}
public void testChunking() {
AbstractChunkedSerializingTestCase.assertChunkCount(createTestInstance(), response -> 2 + response.getIndexToSettings().size());
}