forked from elastic/elasticsearch
-
Notifications
You must be signed in to change notification settings - Fork 1
/
TransportClusterUpdateSettingsAction.java
176 lines (153 loc) · 8.03 KB
/
TransportClusterUpdateSettingsAction.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
/*
* 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.action.admin.cluster.settings;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.MetaData;
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.inject.Inject;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.cluster.ClusterState.newClusterStateBuilder;
/**
*
*/
public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOperationAction<ClusterUpdateSettingsRequest, ClusterUpdateSettingsResponse> {
private final AllocationService allocationService;
private final DynamicSettings dynamicSettings;
@Inject
public TransportClusterUpdateSettingsAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool,
AllocationService allocationService, @ClusterDynamicSettings DynamicSettings dynamicSettings) {
super(settings, transportService, clusterService, threadPool);
this.allocationService = allocationService;
this.dynamicSettings = dynamicSettings;
}
@Override
protected String executor() {
return ThreadPool.Names.MANAGEMENT;
}
@Override
protected String transportAction() {
return ClusterUpdateSettingsAction.NAME;
}
@Override
protected ClusterUpdateSettingsRequest newRequest() {
return new ClusterUpdateSettingsRequest();
}
@Override
protected ClusterUpdateSettingsResponse newResponse() {
return new ClusterUpdateSettingsResponse();
}
@Override
protected ClusterUpdateSettingsResponse masterOperation(final ClusterUpdateSettingsRequest request, ClusterState state) throws ElasticSearchException {
final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
final CountDownLatch latch = new CountDownLatch(1);
clusterService.submitStateUpdateTask("cluster_update_settings", new ProcessedClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
try {
boolean changed = false;
ImmutableSettings.Builder transientSettings = ImmutableSettings.settingsBuilder();
transientSettings.put(currentState.metaData().transientSettings());
for (Map.Entry<String, String> entry : request.transientSettings().getAsMap().entrySet()) {
if (dynamicSettings.hasDynamicSetting(entry.getKey()) || entry.getKey().startsWith("logger.")) {
transientSettings.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring transient setting [{}], not dynamically updateable", entry.getKey());
}
}
ImmutableSettings.Builder persistentSettings = ImmutableSettings.settingsBuilder();
persistentSettings.put(currentState.metaData().persistentSettings());
for (Map.Entry<String, String> entry : request.persistentSettings().getAsMap().entrySet()) {
if (dynamicSettings.hasDynamicSetting(entry.getKey()) || entry.getKey().startsWith("logger.")) {
changed = true;
persistentSettings.put(entry.getKey(), entry.getValue());
} else {
logger.warn("ignoring persistent setting [{}], not dynamically updateable", entry.getKey());
}
}
if (!changed) {
latch.countDown();
return currentState;
}
MetaData.Builder metaData = MetaData.builder().metaData(currentState.metaData())
.persistentSettings(persistentSettings.build())
.transientSettings(transientSettings.build());
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
boolean updatedReadOnly = metaData.persistentSettings().getAsBoolean(MetaData.SETTING_READ_ONLY, false) || metaData.transientSettings().getAsBoolean(MetaData.SETTING_READ_ONLY, false);
if (updatedReadOnly) {
blocks.addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
} else {
blocks.removeGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
}
return ClusterState.builder().state(currentState).metaData(metaData).blocks(blocks).build();
} catch (Exception e) {
latch.countDown();
logger.warn("failed to update cluster settings", e);
return currentState;
} finally {
// we don't release the latch here, only after we rerouted
}
}
@Override
public void clusterStateProcessed(ClusterState clusterState) {
// now, reroute
clusterService.submitStateUpdateTask("reroute_after_cluster_update_settings", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
try {
// now, reroute in case things change that require it (like number of replicas)
RoutingAllocation.Result routingResult = allocationService.reroute(currentState);
return newClusterStateBuilder().state(currentState).routingResult(routingResult).build();
} finally {
latch.countDown();
}
}
});
}
});
try {
latch.await();
} catch (InterruptedException e) {
failureRef.set(e);
}
if (failureRef.get() != null) {
if (failureRef.get() instanceof ElasticSearchException) {
throw (ElasticSearchException) failureRef.get();
} else {
throw new ElasticSearchException(failureRef.get().getMessage(), failureRef.get());
}
}
return new ClusterUpdateSettingsResponse();
}
}