zen disco: support for a node to act as a client (and not become master) using discovery.zen.master setting (default to true). It will automatically be set to false when node.client is set to true.

This commit is contained in:
kimchy 2010-04-25 21:20:24 +03:00
parent 4ab298ce00
commit 265e2fb8bc
8 changed files with 199 additions and 75 deletions

View File

@ -20,21 +20,45 @@
package org.elasticsearch.cluster.node;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.lucene.util.StringHelper;
import org.elasticsearch.util.io.stream.StreamInput;
import org.elasticsearch.util.io.stream.StreamOutput;
import org.elasticsearch.util.io.stream.Streamable;
import org.elasticsearch.util.settings.Settings;
import org.elasticsearch.util.transport.TransportAddress;
import org.elasticsearch.util.transport.TransportAddressSerializers;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;
import static org.elasticsearch.util.transport.TransportAddressSerializers.*;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class DiscoveryNode implements Streamable, Serializable {
public static Map<String, String> buildCommonNodesAttributes(Settings settings) {
Map<String, String> attributes = Maps.newHashMap(settings.getByPrefix("node.").getAsMap());
if (attributes.containsKey("client")) {
if (attributes.get("client").equals("false")) {
attributes.remove("client"); // this is the default
} else {
// if we are client node, don't store data ...
attributes.put("data", "false");
}
}
if (attributes.containsKey("data")) {
if (attributes.get("data").equals("true")) {
attributes.remove("data");
}
}
return attributes;
}
public static final ImmutableList<DiscoveryNode> EMPTY_LIST = ImmutableList.of();
private String nodeName = StringHelper.intern("");
@ -43,22 +67,26 @@ public class DiscoveryNode implements Streamable, Serializable {
private TransportAddress address;
private boolean dataNode = true;
private ImmutableMap<String, String> attributes;
private DiscoveryNode() {
}
public DiscoveryNode(String nodeId, TransportAddress address) {
this("", true, nodeId, address);
this("", nodeId, address, ImmutableMap.<String, String>of());
}
public DiscoveryNode(String nodeName, boolean dataNode, String nodeId, TransportAddress address) {
public DiscoveryNode(String nodeName, String nodeId, TransportAddress address, Map<String, String> attributes) {
if (nodeName == null) {
this.nodeName = StringHelper.intern("");
} else {
this.nodeName = StringHelper.intern(nodeName);
}
this.dataNode = dataNode;
ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
for (Map.Entry<String, String> entry : attributes.entrySet()) {
builder.put(StringHelper.intern(entry.getKey()), StringHelper.intern(entry.getValue()));
}
this.attributes = builder.build();
this.nodeId = StringHelper.intern(nodeId);
this.address = address;
}
@ -105,11 +133,26 @@ public class DiscoveryNode implements Streamable, Serializable {
return name();
}
/**
* The node attributes.
*/
public ImmutableMap<String, String> attributes() {
return this.attributes;
}
/**
* The node attributes.
*/
public ImmutableMap<String, String> getAttributes() {
return attributes();
}
/**
* Should this node hold data (shards) or not.
*/
public boolean dataNode() {
return dataNode;
String data = attributes.get("data");
return data == null || data.equals("true");
}
/**
@ -119,6 +162,18 @@ public class DiscoveryNode implements Streamable, Serializable {
return dataNode();
}
/**
* Is the node a client node or not.
*/
public boolean clientNode() {
String client = attributes.get("client");
return client != null && client.equals("true");
}
public boolean isClientNode() {
return clientNode();
}
public static DiscoveryNode readNode(StreamInput in) throws IOException {
DiscoveryNode node = new DiscoveryNode();
node.readFrom(in);
@ -127,16 +182,25 @@ public class DiscoveryNode implements Streamable, Serializable {
@Override public void readFrom(StreamInput in) throws IOException {
nodeName = StringHelper.intern(in.readUTF());
dataNode = in.readBoolean();
nodeId = StringHelper.intern(in.readUTF());
address = TransportAddressSerializers.addressFromStream(in);
int size = in.readVInt();
ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
for (int i = 0; i < size; i++) {
builder.put(StringHelper.intern(in.readUTF()), StringHelper.intern(in.readUTF()));
}
attributes = builder.build();
}
@Override public void writeTo(StreamOutput out) throws IOException {
out.writeUTF(nodeName);
out.writeBoolean(dataNode);
out.writeUTF(nodeId);
TransportAddressSerializers.addressToStream(out, address);
addressToStream(out, address);
out.writeVInt(attributes.size());
for (Map.Entry<String, String> entry : attributes.entrySet()) {
out.writeUTF(entry.getKey());
out.writeUTF(entry.getValue());
}
}
@Override public boolean equals(Object obj) {
@ -159,12 +223,12 @@ public class DiscoveryNode implements Streamable, Serializable {
if (nodeId != null) {
sb.append('[').append(nodeId).append(']');
}
if (dataNode) {
sb.append("[data]");
}
if (address != null) {
sb.append('[').append(address).append(']');
}
if (!attributes.isEmpty()) {
sb.append(attributes);
}
return sb.toString();
}
}

View File

@ -50,6 +50,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import static com.google.common.collect.Maps.*;
import static com.google.common.collect.Sets.*;
import static org.elasticsearch.cluster.ClusterState.*;
import static org.elasticsearch.cluster.node.DiscoveryNode.*;
/**
* @author kimchy (Shay Banon)
@ -142,11 +143,11 @@ public class JgroupsDiscovery extends AbstractLifecycleComponent<Discovery> impl
channel.connect(clusterName.value());
channel.setReceiver(this);
logger.debug("Connected to cluster [{}], address [{}]", channel.getClusterName(), channel.getAddress());
this.localNode = new DiscoveryNode(settings.get("name"), settings.getAsBoolean("node.data", !settings.getAsBoolean("node.client", false)), channel.getAddress().toString(), transportService.boundAddress().publishAddress());
this.localNode = new DiscoveryNode(settings.get("name"), channel.getAddress().toString(), transportService.boundAddress().publishAddress(), buildCommonNodesAttributes(settings));
if (isMaster()) {
firstMaster = true;
clusterService.submitStateUpdateTask("jgroups-disco-initialconnect(master)", new ProcessedClusterStateUpdateTask() {
clusterService.submitStateUpdateTask("jgroups-disco-initial_connect(master)", new ProcessedClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) {
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder()
.localNodeId(localNode.id())

View File

@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicLong;
import static com.google.common.collect.Sets.*;
import static org.elasticsearch.cluster.ClusterState.*;
import static org.elasticsearch.cluster.node.DiscoveryNode.*;
/**
* @author kimchy (Shay Banon)
@ -84,14 +85,14 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
clusterGroups.put(clusterName, clusterGroup);
}
logger.debug("Connected to cluster [{}]", clusterName);
this.localNode = new DiscoveryNode(settings.get("name"), settings.getAsBoolean("node.data", !settings.getAsBoolean("node.client", false)), Long.toString(nodeIdGenerator.incrementAndGet()), transportService.boundAddress().publishAddress());
this.localNode = new DiscoveryNode(settings.get("name"), Long.toString(nodeIdGenerator.incrementAndGet()), transportService.boundAddress().publishAddress(), buildCommonNodesAttributes(settings));
clusterGroup.members().add(this);
if (clusterGroup.members().size() == 1) {
// we are the first master (and the master)
master = true;
firstMaster = true;
clusterService.submitStateUpdateTask("local-disco-initialconnect(master)", new ProcessedClusterStateUpdateTask() {
clusterService.submitStateUpdateTask("local-disco-initial_connect(master)", new ProcessedClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) {
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder()
.localNodeId(localNode.id())

View File

@ -42,11 +42,13 @@ import org.elasticsearch.util.component.AbstractLifecycleComponent;
import org.elasticsearch.util.settings.Settings;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import static com.google.common.collect.Lists.*;
import static org.elasticsearch.cluster.ClusterState.*;
import static org.elasticsearch.cluster.node.DiscoveryNode.*;
import static org.elasticsearch.cluster.node.DiscoveryNodes.*;
import static org.elasticsearch.util.TimeValue.*;
@ -55,6 +57,8 @@ import static org.elasticsearch.util.TimeValue.*;
*/
public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery, DiscoveryNodesProvider {
private final ThreadPool threadPool;
private final TransportService transportService;
private final ClusterService clusterService;
@ -94,6 +98,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
ZenPingService pingService) {
super(settings);
this.clusterName = clusterName;
this.threadPool = threadPool;
this.clusterService = clusterService;
this.transportService = transportService;
this.pingService = pingService;
@ -114,57 +119,29 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
@Override protected void doStart() throws ElasticSearchException {
localNode = new DiscoveryNode(settings.get("name"), settings.getAsBoolean("node.data", !settings.getAsBoolean("node.client", false)), UUID.randomUUID().toString(), transportService.boundAddress().publishAddress());
Map<String, String> nodeAttributes = buildCommonNodesAttributes(settings);
Boolean zenMaster = componentSettings.getAsBoolean("master", null);
if (zenMaster != null) {
if (zenMaster.equals(Boolean.FALSE)) {
nodeAttributes.put("zen.master", "false");
}
} else if (nodeAttributes.containsKey("client")) {
if (nodeAttributes.get("client").equals("true")) {
nodeAttributes.put("zen.master", "false");
}
}
localNode = new DiscoveryNode(settings.get("name"), UUID.randomUUID().toString(), transportService.boundAddress().publishAddress(), nodeAttributes);
pingService.start();
boolean retry = true;
while (retry) {
retry = false;
DiscoveryNode masterNode = broadBingTillMasterResolved();
if (localNode.equals(masterNode)) {
// we are the master (first)
this.firstMaster = true;
this.master = true;
nodesFD.start(); // start the nodes FD
clusterService.submitStateUpdateTask("zen-disco-initial_connect(master)", new ProcessedClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) {
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder()
.localNodeId(localNode.id())
.masterNodeId(localNode.id())
// put our local node
.put(localNode);
// update the fact that we are the master...
latestDiscoNodes = builder.build();
return newClusterStateBuilder().state(currentState).nodes(builder).build();
}
@Override public void clusterStateProcessed(ClusterState clusterState) {
sendInitialStateEventIfNeeded();
}
});
} else {
this.firstMaster = false;
this.master = false;
try {
// first, make sure we can connect to the master
transportService.connectToNode(masterNode);
} catch (Exception e) {
logger.warn("Failed to connect to master [{}], retrying...", e, masterNode);
retry = true;
continue;
if (nodeAttributes.containsKey("zen.master") && nodeAttributes.get("zen.master").equals("false")) {
// do the join on a different thread
threadPool.execute(new Runnable() {
@Override public void run() {
initialJoin();
}
// send join request
try {
membership.sendJoinRequestBlocking(masterNode, localNode, initialPingTimeout);
} catch (Exception e) {
logger.warn("Failed to send join request to master [{}], retrying...", e, masterNode);
// failed to send the join request, retry
retry = true;
continue;
}
// cool, we found a master, start an FD on it
masterFD.start(masterNode);
}
});
} else {
initialJoin();
}
}
@ -239,6 +216,63 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
publishClusterState.publish(clusterState);
}
private void initialJoin() {
boolean retry = true;
while (retry) {
retry = false;
DiscoveryNode masterNode = broadPingTillMasterResolved();
if (localNode.equals(masterNode)) {
// we are the master (first)
this.firstMaster = true;
this.master = true;
nodesFD.start(); // start the nodes FD
clusterService.submitStateUpdateTask("zen-disco-initial_connect(master)", new ProcessedClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) {
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder()
.localNodeId(localNode.id())
.masterNodeId(localNode.id())
// put our local node
.put(localNode);
// update the fact that we are the master...
latestDiscoNodes = builder.build();
return newClusterStateBuilder().state(currentState).nodes(builder).build();
}
@Override public void clusterStateProcessed(ClusterState clusterState) {
sendInitialStateEventIfNeeded();
}
});
} else {
this.firstMaster = false;
this.master = false;
try {
// first, make sure we can connect to the master
transportService.connectToNode(masterNode);
} catch (Exception e) {
logger.warn("Failed to connect to master [{}], retrying...", e, masterNode);
retry = true;
continue;
}
// send join request
try {
membership.sendJoinRequestBlocking(masterNode, localNode, initialPingTimeout);
} catch (Exception e) {
logger.warn("Failed to send join request to master [{}], retrying...", e, masterNode);
// failed to send the join request, retry
retry = true;
continue;
}
// cool, we found a master, start an FD on it
masterFD.start(masterNode);
}
if (retry) {
if (!lifecycle.started()) {
return;
}
}
}
}
private void handleNodeFailure(final DiscoveryNode node) {
if (!master) {
// nothing to do here...
@ -365,7 +399,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
}
private DiscoveryNode broadBingTillMasterResolved() {
private DiscoveryNode broadPingTillMasterResolved() {
while (true) {
ZenPing.PingResponse[] pingResponses = pingService.pingAndWait(initialPingTimeout);
List<DiscoveryNode> pingMasters = newArrayList();

View File

@ -26,6 +26,7 @@ import org.elasticsearch.util.settings.Settings;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import static com.google.common.collect.Lists.*;
@ -45,7 +46,7 @@ public class ElectMasterService extends AbstractComponent {
* Returns a list of the next possible masters.
*/
public DiscoveryNode[] nextPossibleMasters(Iterable<DiscoveryNode> nodes, int numberOfPossibleMasters) {
List<DiscoveryNode> sortedNodes = sortedNodes(nodes);
List<DiscoveryNode> sortedNodes = sortedMasterNodes(nodes);
if (sortedNodes == null) {
return new DiscoveryNode[0];
}
@ -65,18 +66,27 @@ public class ElectMasterService extends AbstractComponent {
* if no master has been elected.
*/
public DiscoveryNode electMaster(Iterable<DiscoveryNode> nodes) {
List<DiscoveryNode> sortedNodes = sortedNodes(nodes);
if (sortedNodes == null) {
List<DiscoveryNode> sortedNodes = sortedMasterNodes(nodes);
if (sortedNodes == null || sortedNodes.isEmpty()) {
return null;
}
return sortedNodes.get(0);
}
private List<DiscoveryNode> sortedNodes(Iterable<DiscoveryNode> nodes) {
private List<DiscoveryNode> sortedMasterNodes(Iterable<DiscoveryNode> nodes) {
List<DiscoveryNode> possibleNodes = Lists.newArrayList(nodes);
if (possibleNodes.isEmpty()) {
return null;
}
// clean non master nodes
for (Iterator<DiscoveryNode> it = possibleNodes.iterator(); it.hasNext();) {
DiscoveryNode node = it.next();
if (node.attributes().containsKey("zen.master")) {
if (node.attributes().get("zen.master").equals("false")) {
it.remove();
}
}
}
Collections.sort(possibleNodes, nodeComparator);
return possibleNodes;
}

View File

@ -64,7 +64,12 @@ public class RestNodesInfoAction extends BaseRestHandler {
builder.field("name", nodeInfo.node().name());
builder.field("transport_address", nodeInfo.node().address().toString());
builder.field("data_node", nodeInfo.node().dataNode());
builder.startArray("attributes");
for (Map.Entry<String, String> attr : nodeInfo.node().attributes().entrySet()) {
builder.field(attr.getKey(), attr.getValue());
}
builder.endArray();
for (Map.Entry<String, String> nodeAttribute : nodeInfo.attributes().entrySet()) {
builder.field(nodeAttribute.getKey(), nodeAttribute.getValue());

View File

@ -83,15 +83,19 @@ public class ImmutableSettings implements Settings {
throw new SettingsException("Component [" + type + "] does not start with prefix [" + prefix + "]");
}
String settingPrefix = type.substring(prefix.length() + 1); // 1 for the '.'
settingPrefix = settingPrefix.substring(0, settingPrefix.length() - component.getSimpleName().length() - 1); // remove the simple class name
settingPrefix = settingPrefix.substring(0, settingPrefix.length() - component.getSimpleName().length()); // remove the simple class name (keep the dot)
return getByPrefix(settingPrefix);
}
@Override public Settings getByPrefix(String prefix) {
Builder builder = new Builder();
for (Map.Entry<String, String> entry : getAsMap().entrySet()) {
if (entry.getKey().startsWith(settingPrefix)) {
if (entry.getKey().length() <= settingPrefix.length()) {
if (entry.getKey().startsWith(prefix)) {
if (entry.getKey().length() < prefix.length()) {
// ignore this one
continue;
}
builder.put(entry.getKey().substring(settingPrefix.length() + 1), entry.getValue());
builder.put(entry.getKey().substring(prefix.length()), entry.getValue());
}
}
builder.globalSettings(this);

View File

@ -55,7 +55,12 @@ public interface Settings {
Settings getComponentSettings(String prefix, Class component);
/**
* The class loader associted with this settings.
* A settings that are filtered (and key is removed) with the specified prefix.
*/
Settings getByPrefix(String prefix);
/**
* The class loader associated with this settings.
*/
ClassLoader getClassLoader();