# Response for Cluster Settings Update API

If cluster settings are update the REST API returns the accepted values. For
example, updating the `cluster.routing.allocation.disable_allocation` via
cluster settings:

```curl -XPUT http://localhost:9200/_cluster/settings -d '{
    "transient":{
        "cluster.routing.allocation.disable_allocation":"true"
    }
}'```

will respond:

```{
    "persistent":{},
    "transient":{
        "cluster.routing.allocation.disable_allocation":"true"
    }
}```

Closes #2907
This commit is contained in:
Florian Schilling 2013-04-17 17:12:42 +02:00 committed by Simon Willnauer
parent 99c101c37e
commit 54cb4b9615
5 changed files with 161 additions and 5 deletions

View File

@ -19,27 +19,51 @@
package org.elasticsearch.action.admin.cluster.settings;
import java.io.IOException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
/**
* A response for a cluster update settings action.
*/
public class ClusterUpdateSettingsResponse extends ActionResponse {
Settings transientSettings;
Settings persistentSettings;
ClusterUpdateSettingsResponse() {
this.persistentSettings = ImmutableSettings.EMPTY;
this.transientSettings = ImmutableSettings.EMPTY;
}
ClusterUpdateSettingsResponse(Settings transientSettings, Settings persistentSettings) {
this.persistentSettings = persistentSettings;
this.transientSettings = transientSettings;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
transientSettings = ImmutableSettings.readSettingsFromStream(in);
persistentSettings = ImmutableSettings.readSettingsFromStream(in);
}
public Settings getTransientSettings() {
return transientSettings;
}
public Settings getPersistentSettings() {
return persistentSettings;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
ImmutableSettings.writeSettingsToStream(transientSettings, out);
ImmutableSettings.writeSettingsToStream(persistentSettings, out);
}
}

View File

@ -35,6 +35,9 @@ import org.elasticsearch.common.Priority;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -86,6 +89,9 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
final CountDownLatch latch = new CountDownLatch(1);
final ImmutableSettings.Builder transientUpdates = ImmutableSettings.settingsBuilder();
final ImmutableSettings.Builder persistentUpdates = ImmutableSettings.settingsBuilder();
clusterService.submitStateUpdateTask("cluster_update_settings", Priority.URGENT, new ProcessedClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
@ -98,6 +104,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue());
if (error == null) {
transientSettings.put(entry.getKey(), entry.getValue());
transientUpdates.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring transient setting [{}], [{}]", entry.getKey(), error);
@ -114,6 +121,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue());
if (error == null) {
persistentSettings.put(entry.getKey(), entry.getValue());
persistentUpdates.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring persistent setting [{}], [{}]", entry.getKey(), error);
@ -182,6 +190,6 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
}
}
return new ClusterUpdateSettingsResponse();
return new ClusterUpdateSettingsResponse(transientUpdates.build(), persistentUpdates.build());
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.rest;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnicodeUtil;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -54,6 +55,9 @@ public class XContentRestResponse extends AbstractRestResponse {
private final XContentBuilder builder;
public XContentRestResponse(RestRequest request, RestStatus status, XContentBuilder builder) throws IOException {
if(request == null) {
throw new ElasticSearchIllegalArgumentException("request must be set");
}
this.builder = builder;
this.status = status;
this.prefixUtf8Result = startJsonp(request);
@ -149,4 +153,4 @@ public class XContentRestResponse extends AbstractRestResponse {
result.length++;
return result;
}
}
}

View File

@ -26,8 +26,10 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.action.support.RestXContentBuilder;
import java.io.IOException;
import java.util.Map;
@ -67,7 +69,24 @@ public class RestClusterUpdateSettingsAction extends BaseRestHandler {
@Override
public void onResponse(ClusterUpdateSettingsResponse response) {
try {
channel.sendResponse(new StringRestResponse(RestStatus.OK));
XContentBuilder builder = RestXContentBuilder.restContentBuilder(request);
builder.startObject();
builder.startObject("persistent");
for (Map.Entry<String, String> entry : response.getPersistentSettings().getAsMap().entrySet()) {
builder.field(entry.getKey(), entry.getValue());
}
builder.endObject();
builder.startObject("transient");
for (Map.Entry<String, String> entry : response.getTransientSettings().getAsMap().entrySet()) {
builder.field(entry.getKey(), entry.getValue());
}
builder.endObject();
builder.endObject();
channel.sendResponse(new XContentRestResponse(request, RestStatus.OK, builder));
} catch (Throwable e) {
onFailure(e);
}

View File

@ -0,0 +1,101 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.unit.cluster.settings;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.NodeBuilder;
import org.testng.annotations.Test;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
public class ClusterSettingsTests {
@Test
public void clusterSettingsUpdateResponse() {
Node node1 = NodeBuilder.nodeBuilder().node();
Node node2 = NodeBuilder.nodeBuilder().node();
node1.start();
node2.start();
Client client = node1.client();
String key1 = "indices.cache.filter.size";
int value1 = 10;
String key2 = "cluster.routing.allocation.disable_allocation";
boolean value2 = true;
Settings transientSettings1 = ImmutableSettings.builder().put(key1, value1).build();
Settings persistentSettings1 = ImmutableSettings.builder().put(key2, value2).build();
ClusterUpdateSettingsResponse response1 = client.admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(transientSettings1)
.setPersistentSettings(persistentSettings1)
.execute()
.actionGet();
assertThat(response1.getTransientSettings().get(key1), notNullValue());
assertThat(response1.getTransientSettings().get(key2), nullValue());
assertThat(response1.getPersistentSettings().get(key1), nullValue());
assertThat(response1.getPersistentSettings().get(key2), notNullValue());
Settings transientSettings2 = ImmutableSettings.builder().put(key1, value1).put(key2, value2).build();
Settings persistentSettings2 = ImmutableSettings.EMPTY;
ClusterUpdateSettingsResponse response2 = client.admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(transientSettings2)
.setPersistentSettings(persistentSettings2)
.execute()
.actionGet();
assertThat(response2.getTransientSettings().get(key1), notNullValue());
assertThat(response2.getTransientSettings().get(key2), notNullValue());
assertThat(response2.getPersistentSettings().get(key1), nullValue());
assertThat(response2.getPersistentSettings().get(key2), nullValue());
Settings transientSettings3 = ImmutableSettings.EMPTY;
Settings persistentSettings3 = ImmutableSettings.builder().put(key1, value1).put(key2, value2).build();
ClusterUpdateSettingsResponse response3 = client.admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(transientSettings3)
.setPersistentSettings(persistentSettings3)
.execute()
.actionGet();
assertThat(response3.getTransientSettings().get(key1), nullValue());
assertThat(response3.getTransientSettings().get(key2), nullValue());
assertThat(response3.getPersistentSettings().get(key1), notNullValue());
assertThat(response3.getPersistentSettings().get(key2), notNullValue());
node1.stop();
node2.stop();
}
}