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:
parent
ea8fd3b77c
commit
eb4d64f94a
|
@ -0,0 +1,5 @@
|
|||
pr: 126051
|
||||
summary: Run `TransportGetSettingsAction` on local node
|
||||
area: Indices APIs
|
||||
type: enhancement
|
||||
issues: []
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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)"
|
||||
},
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue