allow to filter also by node _name and _id, make sure to reroute properly after cluster update settings
This commit is contained in:
parent
d00edfb165
commit
9ff4a95ee4
|
@ -25,6 +25,7 @@ import org.elasticsearch.action.support.master.TransportMasterNodeOperationActio
|
|||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
||||
|
@ -73,7 +74,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
|
|||
final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
clusterService.submitStateUpdateTask("cluster_update_settings", new ClusterStateUpdateTask() {
|
||||
clusterService.submitStateUpdateTask("cluster_update_settings", new ProcessedClusterStateUpdateTask() {
|
||||
@Override public ClusterState execute(ClusterState currentState) {
|
||||
try {
|
||||
boolean changed = false;
|
||||
|
@ -108,17 +109,30 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeOpe
|
|||
.transientSettings(transientSettings.build());
|
||||
|
||||
|
||||
ClusterState updatedState = ClusterState.builder().state(currentState).metaData(metaData).build();
|
||||
|
||||
// now, reroute in case things change that require it (like number of replicas)
|
||||
RoutingAllocation.Result routingResult = allocationService.reroute(updatedState);
|
||||
updatedState = newClusterStateBuilder().state(updatedState).routingResult(routingResult).build();
|
||||
|
||||
return updatedState;
|
||||
} finally {
|
||||
return ClusterState.builder().state(currentState).metaData(metaData).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 {
|
||||
|
|
|
@ -68,10 +68,25 @@ public class DiscoveryNodeFilters {
|
|||
}
|
||||
InetSocketTransportAddress inetAddress = (InetSocketTransportAddress) node.address();
|
||||
for (String value : values) {
|
||||
if (!Regex.simpleMatch(value, inetAddress.address().getAddress().getHostAddress())) {
|
||||
return false;
|
||||
if (Regex.simpleMatch(value, inetAddress.address().getAddress().getHostAddress())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
} else if ("_id".equals(attr)) {
|
||||
for (String value : values) {
|
||||
if (node.id().equals(value)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
} else if ("_name".equals(attr)) {
|
||||
for (String value : values) {
|
||||
if (Regex.simpleMatch(value, node.name())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
} else {
|
||||
String nodeAttributeValue = node.attributes().get(attr);
|
||||
if (nodeAttributeValue == null) {
|
||||
|
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.integration.cluster.allocation;
|
||||
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.test.integration.AbstractNodesTests;
|
||||
import org.testng.annotations.AfterMethod;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import static org.elasticsearch.common.settings.ImmutableSettings.*;
|
||||
import static org.hamcrest.MatcherAssert.*;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FilteringAllocationTests extends AbstractNodesTests {
|
||||
|
||||
private final ESLogger logger = Loggers.getLogger(FilteringAllocationTests.class);
|
||||
|
||||
@AfterMethod public void cleanAndCloseNodes() throws Exception {
|
||||
closeAllNodes();
|
||||
}
|
||||
|
||||
@Test public void testDecommissionNodeNoReplicas() throws Exception {
|
||||
logger.info("--> starting 2 nodes");
|
||||
startNode("node1");
|
||||
startNode("node2");
|
||||
|
||||
logger.info("--> creating an index with no replicas");
|
||||
client("node1").admin().indices().prepareCreate("test")
|
||||
.setSettings(settingsBuilder().put("index.number_of_replicas", 0))
|
||||
.execute().actionGet();
|
||||
|
||||
ClusterHealthResponse clusterHealthResponse = client("node1").admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet();
|
||||
assertThat(clusterHealthResponse.timedOut(), equalTo(false));
|
||||
|
||||
logger.info("--> index some data");
|
||||
for (int i = 0; i < 100; i++) {
|
||||
client("node1").prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet();
|
||||
}
|
||||
client("node1").admin().indices().prepareRefresh().execute().actionGet();
|
||||
assertThat(client("node1").prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().count(), equalTo(100l));
|
||||
|
||||
logger.info("--> decommission the second node");
|
||||
client("node1").admin().cluster().prepareUpdateSettings()
|
||||
.setTransientSettings(settingsBuilder().put("cluster.routing.allocation.exclude._name", "node2"))
|
||||
.execute().actionGet();
|
||||
|
||||
Thread.sleep(200);
|
||||
|
||||
clusterHealthResponse = client("node1").admin().cluster().prepareHealth()
|
||||
.setWaitForGreenStatus()
|
||||
.setWaitForRelocatingShards(0)
|
||||
.execute().actionGet();
|
||||
assertThat(clusterHealthResponse.timedOut(), equalTo(false));
|
||||
|
||||
logger.info("--> verify all are allocated on node1 now");
|
||||
ClusterState clusterState = client("node1").admin().cluster().prepareState().execute().actionGet().state();
|
||||
for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) {
|
||||
for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) {
|
||||
for (ShardRouting shardRouting : indexShardRoutingTable) {
|
||||
assertThat(clusterState.nodes().get(shardRouting.currentNodeId()).name(), equalTo("node1"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
client("node1").admin().indices().prepareRefresh().execute().actionGet();
|
||||
assertThat(client("node1").prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().count(), equalTo(100l));
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue