Local node master listener

* Fixed an issue where dynamic update to minimum_master_nodes settings would not take immediate effect
* Added LocalNodeMasterListener support to the ClusterService. Enables listening to when the local node becomes/stopped being a master
This commit is contained in:
uboness 2012-08-22 21:48:32 +02:00 committed by Shay Banon
parent 36ff6c9b8b
commit b4b33bb205
6 changed files with 393 additions and 21 deletions

View File

@ -72,6 +72,16 @@ public interface ClusterService extends LifecycleComponent<ClusterService> {
*/
void remove(ClusterStateListener listener);
/**
* Add a listener for on/off local node master events
*/
void add(LocalNodeMasterListener listener);
/**
* Remove the given listener for on/off local master events
*/
void remove(LocalNodeMasterListener listener);
/**
* Adds a cluster state listener that will timeout after the provided timeout.
*/

View File

@ -0,0 +1,52 @@
/*
* 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. 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.cluster;
/**
* Enables listening to master changes events of the local node (when the local node becomes the master, and when the local
* node cease being a master).
*/
public interface LocalNodeMasterListener {
/**
* Called when local node is elected to be the master
*/
void onMaster();
/**
* Called when the local node used to be the master, a new master was elected and it's no longer the local node.
*/
void offMaster();
/**
* The name of the executor that the implementation of the callbacks of this lister should be executed on. The thread
* that is responsible for managing instances of this lister is the same thread handling the cluster state events. If
* the work done is the callbacks above is inexpensive, this value may be {@link org.elasticsearch.threadpool.ThreadPool.Names#SAME SAME}
* (indicating that the callbaks will run on the same thread as the cluster state events are fired with). On the other hand,
* if the logic in the callbacks are heavier and take longer to process (or perhaps involve blocking due to IO operations),
* prefer to execute them on a separte more appropriate executor (eg. {@link org.elasticsearch.threadpool.ThreadPool.Names#GENERIC GENERIC}
* or {@link org.elasticsearch.threadpool.ThreadPool.Names#MANAGEMENT MANAGEMENT}).
*
* @return The name of the executor that will run the callbacks of this listener.
*/
String executorName();
}

View File

@ -43,10 +43,7 @@ import org.elasticsearch.transport.TransportService;
import java.util.Iterator;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.*;
import static java.util.concurrent.Executors.newSingleThreadExecutor;
import static org.elasticsearch.cluster.ClusterState.Builder;
@ -75,6 +72,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private final List<ClusterStateListener> priorityClusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
private final List<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
private final List<ClusterStateListener> lastClusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
private final LocalNodeMasterListeners localNodeMasterListeners;
private final Queue<NotifyTimeout> onGoingTimeouts = ConcurrentCollections.newQueue();
@ -97,6 +95,8 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
this.nodeSettingsService.setClusterService(this);
this.reconnectInterval = componentSettings.getAsTime("reconnect_interval", TimeValue.timeValueSeconds(10));
localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
}
public NodeSettingsService settingsService() {
@ -112,6 +112,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
@Override
protected void doStart() throws ElasticSearchException {
add(localNodeMasterListeners);
this.clusterState = newClusterStateBuilder().blocks(initialBlocks).build();
this.updateTasksExecutor = newSingleThreadExecutor(daemonThreadFactory(settings, "clusterService#updateTask"));
this.reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, new ReconnectToNodes());
@ -130,6 +131,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
} catch (InterruptedException e) {
// ignore
}
remove(localNodeMasterListeners);
}
@Override
@ -175,6 +177,16 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
}
@Override
public void add(LocalNodeMasterListener listener) {
localNodeMasterListeners.add(listener);
}
@Override
public void remove(LocalNodeMasterListener listener) {
localNodeMasterListeners.remove(listener);
}
public void add(TimeValue timeout, final TimeoutClusterStateListener listener) {
if (lifecycle.stoppedOrClosed()) {
listener.onClose();
@ -398,4 +410,75 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private boolean nodeRequiresConnection(DiscoveryNode node) {
return localNode().shouldConnectTo(node);
}
private static class LocalNodeMasterListeners implements ClusterStateListener {
private final List<LocalNodeMasterListener> listeners = new CopyOnWriteArrayList<LocalNodeMasterListener>();
private final ThreadPool threadPool;
private volatile boolean master = false;
private LocalNodeMasterListeners(ThreadPool threadPool) {
this.threadPool = threadPool;
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
if (!master && event.localNodeMaster()) {
master = true;
for (LocalNodeMasterListener listener : listeners) {
Executor executor = threadPool.executor(listener.executorName());
executor.execute(new OnMasterRunnable(listener));
}
return;
}
if (master && !event.localNodeMaster()) {
master = false;
for (LocalNodeMasterListener listener : listeners) {
Executor executor = threadPool.executor(listener.executorName());
executor.execute(new OffMasterRunnable(listener));
}
}
}
private void add(LocalNodeMasterListener listener) {
listeners.add(listener);
}
private void remove(LocalNodeMasterListener listener) {
listeners.remove(listener);
}
private void clear() {
listeners.clear();
}
}
private static class OnMasterRunnable implements Runnable {
private final LocalNodeMasterListener listener;
private OnMasterRunnable(LocalNodeMasterListener listener) {
this.listener = listener;
}
@Override
public void run() {
listener.onMaster();
}
}
private static class OffMasterRunnable implements Runnable {
private final LocalNodeMasterListener listener;
private OffMasterRunnable(LocalNodeMasterListener listener) {
this.listener = listener;
}
@Override
public void run() {
listener.offMaster();
}
}
}

View File

@ -142,7 +142,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
logger.debug("using ping.timeout [{}], master_election.filter_client [{}], master_election.filter_data [{}]", pingTimeout, masterElectionFilterClientNodes, masterElectionFilterDataNodes);
this.electMaster = new ElectMasterService(settings, nodeSettingsService);
this.electMaster = new ElectMasterService(settings);
nodeSettingsService.addListener(new ApplySettings());
this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, this);
this.masterFD.addListener(new MasterNodeFailureListener());
@ -410,6 +411,34 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
});
}
private void handleMinimumMasterNodesChanged(final int minimumMasterNodes) {
if (lifecycleState() != Lifecycle.State.STARTED) {
// not started, ignore a node failure
return;
}
if (!master) {
// nothing to do here...
return;
}
clusterService.submitStateUpdateTask("zen-disco-minimum_master_nodes_changed", new ProcessedClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
final int prevMinimumMasterNode = ZenDiscovery.this.electMaster.minimumMasterNodes();
ZenDiscovery.this.electMaster.minimumMasterNodes(minimumMasterNodes);
// check if we have enough master nodes, if not, we need to move into joining the cluster again
if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
return rejoin(currentState, "not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]");
}
return currentState;
}
@Override
public void clusterStateProcessed(ClusterState clusterState) {
sendInitialStateEventIfNeeded();
}
});
}
private void handleMasterGone(final DiscoveryNode masterNode, final String reason) {
if (lifecycleState() != Lifecycle.State.STARTED) {
// not started, ignore a master failure
@ -754,12 +783,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override
public void readFrom(StreamInput in) throws IOException {
fromNodeId = in.readOptionalUTF();
fromNodeId = in.readOptionalString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalUTF(fromNodeId);
out.writeOptionalString(fromNodeId);
}
}
@ -793,4 +822,15 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int minimumMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", ZenDiscovery.this.electMaster.minimumMasterNodes());
if (minimumMasterNodes != ZenDiscovery.this.electMaster.minimumMasterNodes()) {
logger.info("updating discovery.zen.minimum_master_nodes from [{}] to [{}]", ZenDiscovery.this.electMaster.minimumMasterNodes(), minimumMasterNodes);
handleMinimumMasterNodesChanged(minimumMasterNodes);
}
}
}
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Collections;
import java.util.Comparator;
@ -44,11 +43,18 @@ public class ElectMasterService extends AbstractComponent {
private volatile int minimumMasterNodes;
public ElectMasterService(Settings settings, NodeSettingsService nodeSettingsService) {
public ElectMasterService(Settings settings) {
super(settings);
this.minimumMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", -1);
logger.debug("using minimum_master_nodes [{}]", minimumMasterNodes);
nodeSettingsService.addListener(new ApplySettings());
}
public void minimumMasterNodes(int minimumMasterNodes) {
this.minimumMasterNodes = minimumMasterNodes;
}
public int minimumMasterNodes() {
return minimumMasterNodes;
}
public boolean hasEnoughMasterNodes(Iterable<DiscoveryNode> nodes) {
@ -111,17 +117,6 @@ public class ElectMasterService extends AbstractComponent {
return possibleNodes;
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int minimumMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", ElectMasterService.this.minimumMasterNodes);
if (minimumMasterNodes != ElectMasterService.this.minimumMasterNodes) {
logger.info("updating [discovery.zen.minimum_master_nodes] from [{}] to [{}]", ElectMasterService.this.minimumMasterNodes, minimumMasterNodes);
ElectMasterService.this.minimumMasterNodes = minimumMasterNodes;
}
}
}
private static class NodeComparator implements Comparator<DiscoveryNode> {
@Override

View File

@ -0,0 +1,192 @@
/*
* 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. 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.integration.cluster;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.LocalNodeMasterListener;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.Singleton;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.internal.InternalNode;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.threadpool.ThreadPool;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.Test;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.*;
/**
*
*/
public class LocalNodeMasterListenerTests extends AbstractZenNodesTests {
@AfterMethod
public void closeNodes() {
closeAllNodes();
}
@Test
public void testListenerCallbacks() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.zen.minimum_master_nodes", 1)
.put("discovery.zen.ping_timeout", "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put("plugin.types", TestPlugin.class.getName())
.build();
InternalNode node1 = (InternalNode) startNode("node1", settings);
ClusterService clusterService1 = node1.injector().getInstance(ClusterService.class);
MasterAwareService testService1 = node1.injector().getInstance(MasterAwareService.class);
// the first node should be a master as the minimum required is 1
assertThat(clusterService1.state().nodes().masterNode(), notNullValue());
assertThat(clusterService1.state().nodes().localNodeMaster(), is(true));
assertThat(testService1.master(), is(true));
InternalNode node2 = (InternalNode) startNode("node2", settings);
ClusterService clusterService2 = node2.injector().getInstance(ClusterService.class);
MasterAwareService testService2 = node2.injector().getInstance(MasterAwareService.class);
ClusterHealthResponse clusterHealth = node2.client().admin().cluster().prepareHealth().setWaitForNodes("2").execute().actionGet();
assertThat(clusterHealth.timedOut(), equalTo(false));
// the second node should not be the master as node1 is already the master.
assertThat(clusterService2.state().nodes().localNodeMaster(), is(false));
assertThat(testService2.master(), is(false));
node1.close();
clusterHealth = node2.client().admin().cluster().prepareHealth().setWaitForNodes("1").execute().actionGet();
assertThat(clusterHealth.timedOut(), equalTo(false));
// now that node1 is closed, node2 should be elected as master
assertThat(clusterService2.state().nodes().localNodeMaster(), is(true));
assertThat(testService2.master(), is(true));
Settings newSettings = settingsBuilder()
.put("discovery.zen.minimum_master_nodes", 2)
.build();
node2.client().admin().cluster().prepareUpdateSettings().setTransientSettings(newSettings).execute().actionGet();
Thread.sleep(200);
// there should not be any master as the minimum number of required eligible masters is not met
assertThat(clusterService2.state().nodes().masterNode(), is(nullValue()));
assertThat(testService2.master(), is(false));
node1 = (InternalNode) startNode("node1", settings);
clusterService1 = node1.injector().getInstance(ClusterService.class);
testService1 = node1.injector().getInstance(MasterAwareService.class);
clusterHealth = node2.client().admin().cluster().prepareHealth().setWaitForNodes("2").execute().actionGet();
assertThat(clusterHealth.timedOut(), equalTo(false));
// now that we started node1 again, a new master should be elected
assertThat(clusterService1.state().nodes().masterNode(), is(notNullValue()));
if ("node1".equals(clusterService1.state().nodes().masterNode().name())) {
assertThat(testService1.master(), is(true));
assertThat(testService2.master(), is(false));
} else {
assertThat(testService1.master(), is(false));
assertThat(testService2.master(), is(true));
}
}
public static class TestPlugin extends AbstractPlugin {
@Override
public String name() {
return "test plugin";
}
@Override
public String description() {
return "test plugin";
}
@Override
public Collection<Class<? extends LifecycleComponent>> services() {
List<Class<? extends LifecycleComponent>> services = new ArrayList<Class<? extends LifecycleComponent>>(1);
services.add(MasterAwareService.class);
return services;
}
}
@Singleton
public static class MasterAwareService extends AbstractLifecycleComponent<MasterAwareService> implements LocalNodeMasterListener {
private final ClusterService clusterService;
private volatile boolean master;
@Inject
public MasterAwareService(Settings settings, ClusterService clusterService) {
super(settings);
clusterService.add(this);
this.clusterService = clusterService;
logger.info("initialized test service");
}
@Override
public void onMaster() {
logger.info("on master [" + clusterService.state().nodes().localNode() + "]");
master = true;
}
@Override
public void offMaster() {
logger.info("off master [" + clusterService.state().nodes().localNode() + "]");
master = false;
}
public boolean master() {
return master;
}
@Override
protected void doStart() throws ElasticSearchException {
}
@Override
protected void doStop() throws ElasticSearchException {
}
@Override
protected void doClose() throws ElasticSearchException {
}
@Override
public String executorName() {
return ThreadPool.Names.SAME;
}
}
}