Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Faster GET _cluster/settings API #86405

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions docs/changelog/86405.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
pr: 86405
summary: Faster GET _cluster/settings API
area: Infra/Core
type: enhancement
issues:
- 82342
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,9 @@
import org.elasticsearch.action.admin.cluster.repositories.verify.VerifyRepositoryAction;
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteAction;
import org.elasticsearch.action.admin.cluster.reroute.TransportClusterRerouteAction;
import org.elasticsearch.action.admin.cluster.settings.ClusterGetSettingsAction;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsAction;
import org.elasticsearch.action.admin.cluster.settings.TransportClusterGetSettingsAction;
import org.elasticsearch.action.admin.cluster.settings.TransportClusterUpdateSettingsAction;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction;
import org.elasticsearch.action.admin.cluster.shards.TransportClusterSearchShardsAction;
Expand Down Expand Up @@ -547,6 +549,7 @@ public <Request extends ActionRequest, Response extends ActionResponse> void reg
actions.register(ClusterStateAction.INSTANCE, TransportClusterStateAction.class);
actions.register(ClusterHealthAction.INSTANCE, TransportClusterHealthAction.class);
actions.register(ClusterUpdateSettingsAction.INSTANCE, TransportClusterUpdateSettingsAction.class);
actions.register(ClusterGetSettingsAction.INSTANCE, TransportClusterGetSettingsAction.class);
actions.register(ClusterRerouteAction.INSTANCE, TransportClusterRerouteAction.class);
actions.register(ClusterSearchShardsAction.INSTANCE, TransportClusterSearchShardsAction.class);
actions.register(PendingClusterTasksAction.INSTANCE, TransportPendingClusterTasksAction.class);
Expand Down Expand Up @@ -709,7 +712,7 @@ public void initRestHandlers(Supplier<DiscoveryNodes> nodesInCluster) {
registerHandler.accept(new RestClusterStateAction(settingsFilter, threadPool));
registerHandler.accept(new RestClusterHealthAction());
registerHandler.accept(new RestClusterUpdateSettingsAction());
registerHandler.accept(new RestClusterGetSettingsAction(settings, clusterSettings, settingsFilter));
registerHandler.accept(new RestClusterGetSettingsAction(settings, clusterSettings, settingsFilter, nodesInCluster));
registerHandler.accept(new RestClusterRerouteAction(settingsFilter));
registerHandler.accept(new RestClusterSearchShardsAction());
registerHandler.accept(new RestPendingClusterTasksAction());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.action.admin.cluster.settings;

import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;

import java.io.IOException;
import java.util.Objects;

public class ClusterGetSettingsAction extends ActionType<ClusterGetSettingsAction.Response> {

public static final ClusterGetSettingsAction INSTANCE = new ClusterGetSettingsAction();
public static final String NAME = "cluster:monitor/settings";

public ClusterGetSettingsAction() {
super(NAME, Response::new);
}

/**
* Request to retrieve the cluster settings
*/
public static class Request extends MasterNodeReadRequest<Request> {
public Request() {}

public Request(StreamInput in) throws IOException {
super(in);
assert in.getVersion().onOrAfter(Version.V_8_3_0);
}
DaveCTurner marked this conversation as resolved.
Show resolved Hide resolved

@Override
public void writeTo(StreamOutput out) throws IOException {
assert out.getVersion().onOrAfter(Version.V_8_3_0);
super.writeTo(out);
}

@Override
public ActionRequestValidationException validate() {
return null;
}
}

/**
* Response for cluster settings
*/
public static class Response extends ActionResponse {
private final Settings persistentSettings;
private final Settings transientSettings;
private final Settings settings;

@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Response response = (Response) o;
return Objects.equals(persistentSettings, response.persistentSettings)
&& Objects.equals(transientSettings, response.transientSettings)
&& Objects.equals(settings, response.settings);
}

@Override
public int hashCode() {
return Objects.hash(persistentSettings, transientSettings, settings);
}

public Response(StreamInput in) throws IOException {
super(in);
assert in.getVersion().onOrAfter(Version.V_8_3_0);
persistentSettings = Settings.readSettingsFromStream(in);
transientSettings = Settings.readSettingsFromStream(in);
settings = Settings.readSettingsFromStream(in);
}

public Response(Settings persistentSettings, Settings transientSettings, Settings settings) {
this.persistentSettings = Objects.requireNonNullElse(persistentSettings, Settings.EMPTY);
this.transientSettings = Objects.requireNonNullElse(transientSettings, Settings.EMPTY);
this.settings = Objects.requireNonNullElse(settings, Settings.EMPTY);
}

@Override
public void writeTo(StreamOutput out) throws IOException {
assert out.getVersion().onOrAfter(Version.V_8_3_0);
Settings.writeSettingsToStream(persistentSettings, out);
Settings.writeSettingsToStream(transientSettings, out);
Settings.writeSettingsToStream(settings, out);
}

public Settings persistentSettings() {
return persistentSettings;
}

public Settings transientSettings() {
return transientSettings;
}

public Settings settings() {
return settings;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.action.admin.cluster.settings;

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.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;

public class TransportClusterGetSettingsAction extends TransportMasterNodeReadAction<
ClusterGetSettingsAction.Request,
ClusterGetSettingsAction.Response> {

private final SettingsFilter settingsFilter;

@Inject
public TransportClusterGetSettingsAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
SettingsFilter settingsFilter,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver
) {
super(
ClusterGetSettingsAction.NAME,
false,
transportService,
clusterService,
threadPool,
actionFilters,
ClusterGetSettingsAction.Request::new,
indexNameExpressionResolver,
ClusterGetSettingsAction.Response::new,
ThreadPool.Names.SAME
);

this.settingsFilter = settingsFilter;
}

@Override
protected void masterOperation(
Task task,
ClusterGetSettingsAction.Request request,
ClusterState state,
ActionListener<ClusterGetSettingsAction.Response> listener
) throws Exception {
Metadata metadata = state.metadata();
listener.onResponse(
new ClusterGetSettingsAction.Response(
settingsFilter.filter(metadata.persistentSettings()),
settingsFilter.filter(metadata.transientSettings()),
settingsFilter.filter(metadata.settings())
)
);
}

@Override
protected ClusterBlockException checkBlock(ClusterGetSettingsAction.Request request, ClusterState state) {
return null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,14 @@

package org.elasticsearch.rest.action.admin.cluster;

import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.settings.ClusterGetSettingsAction;
import org.elasticsearch.action.admin.cluster.settings.RestClusterGetSettingsResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
import org.elasticsearch.client.internal.Requests;
import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
Expand All @@ -23,6 +26,7 @@
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.function.Supplier;

import static org.elasticsearch.rest.RestRequest.Method.GET;

Expand All @@ -31,11 +35,18 @@ public class RestClusterGetSettingsAction extends BaseRestHandler {
private final Settings settings;
private final ClusterSettings clusterSettings;
private final SettingsFilter settingsFilter;
private final Supplier<DiscoveryNodes> nodesInCluster;

public RestClusterGetSettingsAction(Settings settings, ClusterSettings clusterSettings, SettingsFilter settingsFilter) {
public RestClusterGetSettingsAction(
Settings settings,
ClusterSettings clusterSettings,
SettingsFilter settingsFilter,
Supplier<DiscoveryNodes> nodesInCluster
) {
this.settings = settings;
this.clusterSettings = clusterSettings;
this.settingsFilter = settingsFilter;
this.nodesInCluster = nodesInCluster;
}

@Override
Expand All @@ -48,18 +59,51 @@ public String getName() {
return "cluster_get_settings_action";
}

private void setUpRequestParams(MasterNodeReadRequest<?> clusterRequest, RestRequest request) {
clusterRequest.local(request.paramAsBoolean("local", clusterRequest.local()));
clusterRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterRequest.masterNodeTimeout()));
}

@Override
public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException {
ClusterStateRequest clusterStateRequest = Requests.clusterStateRequest().routingTable(false).nodes(false);
final boolean renderDefaults = request.paramAsBoolean("include_defaults", false);
clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local()));
clusterStateRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterStateRequest.masterNodeTimeout()));

if (nodesInCluster.get().getMinNodeVersion().before(Version.V_8_3_0)) {
return prepareLegacyRequest(request, client, renderDefaults);
}

ClusterGetSettingsAction.Request clusterSettingsRequest = new ClusterGetSettingsAction.Request();

setUpRequestParams(clusterSettingsRequest, request);

return channel -> client.execute(
ClusterGetSettingsAction.INSTANCE,
clusterSettingsRequest,
new RestToXContentListener<RestClusterGetSettingsResponse>(channel).map(
r -> response(r, renderDefaults, settingsFilter, clusterSettings, settings)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd still sort of like to reduce the duplication here between the two branches but I see it's not so simple because we have to consume the request params before we get hold of the channel. Not a blocking suggestion.

)
);
}

private RestChannelConsumer prepareLegacyRequest(final RestRequest request, final NodeClient client, final boolean renderDefaults) {
ClusterStateRequest clusterStateRequest = Requests.clusterStateRequest().routingTable(false).nodes(false);
setUpRequestParams(clusterStateRequest, request);
return channel -> client.admin()
.cluster()
.state(
clusterStateRequest,
new RestToXContentListener<RestClusterGetSettingsResponse>(channel).map(
response -> response(response.getState(), renderDefaults, settingsFilter, clusterSettings, settings)
r -> response(
new ClusterGetSettingsAction.Response(
r.getState().metadata().persistentSettings(),
r.getState().metadata().transientSettings(),
r.getState().metadata().settings()
),
renderDefaults,
settingsFilter,
clusterSettings,
settings
)
)
);
}
Expand All @@ -75,16 +119,16 @@ public boolean canTripCircuitBreaker() {
}

static RestClusterGetSettingsResponse response(
final ClusterState state,
final ClusterGetSettingsAction.Response response,
final boolean renderDefaults,
final SettingsFilter settingsFilter,
final ClusterSettings clusterSettings,
final Settings settings
) {
return new RestClusterGetSettingsResponse(
settingsFilter.filter(state.metadata().persistentSettings()),
settingsFilter.filter(state.metadata().transientSettings()),
renderDefaults ? settingsFilter.filter(clusterSettings.diff(state.metadata().settings(), settings)) : Settings.EMPTY
settingsFilter.filter(response.persistentSettings()),
settingsFilter.filter(response.transientSettings()),
renderDefaults ? settingsFilter.filter(clusterSettings.diff(response.settings(), settings)) : Settings.EMPTY
);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.action.admin.cluster.settings;

import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.AbstractWireSerializingTestCase;

public class ClusterGetSettingsSerializationTests extends AbstractWireSerializingTestCase<ClusterGetSettingsAction.Response> {
@Override
protected Writeable.Reader<ClusterGetSettingsAction.Response> instanceReader() {
return ClusterGetSettingsAction.Response::new;
}

@Override
protected ClusterGetSettingsAction.Response createTestInstance() {
final Settings persistentSettings = Settings.builder()
.put("persistent.foo.filtered", "bar")
.put("persistent.foo.non_filtered", "baz")
.build();

final Settings transientSettings = Settings.builder()
.put("transient.foo.filtered", "bar")
.put("transient.foo.non_filtered", "baz")
.build();

final Settings allSettings = Settings.builder().put(persistentSettings).put(transientSettings).build();

return new ClusterGetSettingsAction.Response(persistentSettings, transientSettings, allSettings);
}

@Override
protected ClusterGetSettingsAction.Response mutateInstance(ClusterGetSettingsAction.Response instance) {
final Settings otherSettings = Settings.builder().put("random.setting", randomAlphaOfLength(randomIntBetween(1, 12))).build();
return switch (between(0, 2)) {
case 0 -> new ClusterGetSettingsAction.Response(otherSettings, instance.transientSettings(), instance.settings());
case 1 -> new ClusterGetSettingsAction.Response(instance.persistentSettings(), otherSettings, instance.settings());
case 2 -> new ClusterGetSettingsAction.Response(instance.persistentSettings(), instance.transientSettings(), otherSettings);
default -> throw new IllegalStateException("Unexpected switch value");
};
}
}
Loading