|
@@ -22,16 +22,17 @@ package org.elasticsearch.action.admin.cluster.settings;
|
|
|
import org.elasticsearch.ElasticSearchException;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction;
|
|
|
+import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
|
|
import org.elasticsearch.cluster.ClusterService;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
-import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|
|
-import org.elasticsearch.cluster.TimeoutClusterStateUpdateTask;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
|
|
import org.elasticsearch.cluster.metadata.MetaData;
|
|
|
+import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
|
|
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
|
|
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
|
|
|
import org.elasticsearch.cluster.settings.DynamicSettings;
|
|
|
+import org.elasticsearch.common.Nullable;
|
|
|
import org.elasticsearch.common.Priority;
|
|
|
import org.elasticsearch.common.inject.Inject;
|
|
|
import org.elasticsearch.common.settings.ImmutableSettings;
|
|
@@ -42,6 +43,7 @@ import org.elasticsearch.transport.TransportService;
|
|
|
|
|
|
import java.util.Map;
|
|
|
|
|
|
+import static org.elasticsearch.cluster.ClusterState.builder;
|
|
|
import static org.elasticsearch.cluster.ClusterState.newClusterStateBuilder;
|
|
|
|
|
|
/**
|
|
@@ -86,7 +88,91 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
|
|
|
final ImmutableSettings.Builder transientUpdates = ImmutableSettings.settingsBuilder();
|
|
|
final ImmutableSettings.Builder persistentUpdates = ImmutableSettings.settingsBuilder();
|
|
|
|
|
|
- clusterService.submitStateUpdateTask("cluster_update_settings", Priority.URGENT, new TimeoutClusterStateUpdateTask() {
|
|
|
+ clusterService.submitStateUpdateTask("cluster_update_settings", Priority.URGENT, new AckedClusterStateUpdateTask() {
|
|
|
+
|
|
|
+ private volatile boolean changed = false;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean mustAck(DiscoveryNode discoveryNode) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onAllNodesAcked(@Nullable Throwable t) {
|
|
|
+ if (changed) {
|
|
|
+ reroute(true);
|
|
|
+ } else {
|
|
|
+ listener.onResponse(new ClusterUpdateSettingsResponse(true, transientUpdates.build(), persistentUpdates.build()));
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onAckTimeout() {
|
|
|
+ if (changed) {
|
|
|
+ reroute(false);
|
|
|
+ } else {
|
|
|
+ listener.onResponse(new ClusterUpdateSettingsResponse(false, transientUpdates.build(), persistentUpdates.build()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void reroute(final boolean updateSettingsAcked) {
|
|
|
+ clusterService.submitStateUpdateTask("reroute_after_cluster_update_settings", Priority.URGENT, new AckedClusterStateUpdateTask() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean mustAck(DiscoveryNode discoveryNode) {
|
|
|
+ //we wait for the reroute ack only if the update settings was acknowledged
|
|
|
+ return updateSettingsAcked;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onAllNodesAcked(@Nullable Throwable t) {
|
|
|
+ //we return when the cluster reroute is acked (the acknowledged flag depends on whether the update settings was acknowledged)
|
|
|
+ listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, transientUpdates.build(), persistentUpdates.build()));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onAckTimeout() {
|
|
|
+ //we return when the cluster reroute ack times out (acknowledged false)
|
|
|
+ listener.onResponse(new ClusterUpdateSettingsResponse(false, transientUpdates.build(), persistentUpdates.build()));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public TimeValue ackTimeout() {
|
|
|
+ return request.timeout();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public TimeValue timeout() {
|
|
|
+ return request.masterNodeTimeout();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(String source, Throwable t) {
|
|
|
+ //if the reroute fails we only log
|
|
|
+ logger.debug("failed to perform [{}]", t, source);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public ClusterState execute(final ClusterState currentState) {
|
|
|
+ // now, reroute in case things that require it changed (e.g. number of replicas)
|
|
|
+ RoutingAllocation.Result routingResult = allocationService.reroute(currentState);
|
|
|
+ if (!routingResult.changed()) {
|
|
|
+ return currentState;
|
|
|
+ }
|
|
|
+ return newClusterStateBuilder().state(currentState).routingResult(routingResult).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public TimeValue ackTimeout() {
|
|
|
+ return request.timeout();
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
public TimeValue timeout() {
|
|
@@ -101,7 +187,6 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
|
|
|
|
|
|
@Override
|
|
|
public ClusterState execute(final ClusterState currentState) {
|
|
|
- boolean changed = false;
|
|
|
ImmutableSettings.Builder transientSettings = ImmutableSettings.settingsBuilder();
|
|
|
transientSettings.put(currentState.metaData().transientSettings());
|
|
|
for (Map.Entry<String, String> entry : request.transientSettings().getAsMap().entrySet()) {
|
|
@@ -152,38 +237,12 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
|
|
|
blocks.removeGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
|
|
|
}
|
|
|
|
|
|
- return ClusterState.builder().state(currentState).metaData(metaData).blocks(blocks).build();
|
|
|
+ return builder().state(currentState).metaData(metaData).blocks(blocks).build();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
- if (oldState == newState) {
|
|
|
- // nothing changed...
|
|
|
- listener.onResponse(new ClusterUpdateSettingsResponse(transientUpdates.build(), persistentUpdates.build()));
|
|
|
- return;
|
|
|
- }
|
|
|
- // now, reroute
|
|
|
- clusterService.submitStateUpdateTask("reroute_after_cluster_update_settings", Priority.URGENT, new ClusterStateUpdateTask() {
|
|
|
- @Override
|
|
|
- public ClusterState execute(final ClusterState currentState) {
|
|
|
- try {
|
|
|
- // now, reroute in case things change that require it (like number of replicas)
|
|
|
- RoutingAllocation.Result routingResult = allocationService.reroute(currentState);
|
|
|
- if (!routingResult.changed()) {
|
|
|
- return currentState;
|
|
|
- }
|
|
|
- return newClusterStateBuilder().state(currentState).routingResult(routingResult).build();
|
|
|
- } finally {
|
|
|
- listener.onResponse(new ClusterUpdateSettingsResponse(transientUpdates.build(), persistentUpdates.build()));
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- @Override
|
|
|
- public void onFailure(String source, Throwable t) {
|
|
|
- logger.warn("unexpected failure during [{}]", t, source);
|
|
|
- listener.onResponse(new ClusterUpdateSettingsResponse(transientUpdates.build(), persistentUpdates.build()));
|
|
|
- }
|
|
|
- });
|
|
|
}
|
|
|
});
|
|
|
}
|