Added generic cluster state update ack mechanism
Added new AckedClusterStateUpdateTask interface that can be used to submit cluster state update tasks and allows actions to be notified back when a set of (configurable) nodes have acknowledged the cluster state update. Supports a configurable timeout, so that we wait for acknowledgement for a limited amount of time (will be provided in the request as it curently happens, default 10s). Internally, a low level AckListener is created (InternalClusterService) and passed to the publish method, so that it can be notified whenever each node responds to the publish request. Once all the expected nodes have responded or the timeoeout has expired, the AckListener notifies the action which will return adding the proper acknowledged flag to the response. Ideally, this new mechanism will gradually replace the existing ones based on custom endpoints and notifications (per api). Closes #3786
This commit is contained in:
parent
f5e2cf9785
commit
55f1eab09a
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
/**
|
||||
* An extension interface to {@link ClusterStateUpdateTask} that allows to be notified when
|
||||
* all the nodes have acknowledged a cluster state update request
|
||||
*/
|
||||
public interface AckedClusterStateUpdateTask extends TimeoutClusterStateUpdateTask {
|
||||
|
||||
/**
|
||||
* Called to determine which nodes the acknowledgement is expected from
|
||||
* @param discoveryNode a node
|
||||
* @return true if the node is expected to send ack back, false otherwise
|
||||
*/
|
||||
boolean mustAck(DiscoveryNode discoveryNode);
|
||||
|
||||
/**
|
||||
* Called once all the nodes have acknowledged the cluster state update request. Must be
|
||||
* very lightweight execution, since it gets executed on the cluster service thread.
|
||||
* @param t optional error that might have been thrown
|
||||
*/
|
||||
void onAllNodesAcked(@Nullable Throwable t);
|
||||
|
||||
/**
|
||||
* Called once the acknowledgement timeout defined by
|
||||
* {@link AckedClusterStateUpdateTask#ackTimeout()} has expired
|
||||
*/
|
||||
void onAckTimeout();
|
||||
|
||||
/**
|
||||
* Acknowledgement timeout, maximum time interval to wait for acknowledgements
|
||||
*/
|
||||
TimeValue ackTimeout();
|
||||
}
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
|
|||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.routing.operation.OperationRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
|
@ -49,6 +50,8 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.cluster.ClusterState.newClusterStateBuilder;
|
||||
import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory;
|
||||
|
@ -317,6 +320,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
}
|
||||
|
||||
try {
|
||||
Discovery.AckListener ackListener = new NoOpAckListener();
|
||||
if (newClusterState.nodes().localNodeMaster()) {
|
||||
// only the master controls the version numbers
|
||||
Builder builder = ClusterState.builder().state(newClusterState).version(newClusterState.version() + 1);
|
||||
|
@ -327,6 +331,19 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
builder.metaData(MetaData.builder().metaData(newClusterState.metaData()).version(newClusterState.metaData().version() + 1));
|
||||
}
|
||||
newClusterState = builder.build();
|
||||
|
||||
if (updateTask instanceof AckedClusterStateUpdateTask) {
|
||||
final AckedClusterStateUpdateTask ackedUpdateTask = (AckedClusterStateUpdateTask) updateTask;
|
||||
try {
|
||||
ackListener = new AckCountDownListener(ackedUpdateTask, newClusterState.version(), newClusterState.nodes(), threadPool);
|
||||
} catch(EsRejectedExecutionException ex) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Couldn't schedule timeout thread - node might be shutting down", ex);
|
||||
}
|
||||
//timeout straightaway, otherwise we could wait forever as the timeout thread has not started
|
||||
ackedUpdateTask.onAckTimeout();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (previousClusterState.blocks().hasGlobalBlock(Discovery.NO_MASTER_BLOCK) && !newClusterState.blocks().hasGlobalBlock(Discovery.NO_MASTER_BLOCK)) {
|
||||
// force an update, its a fresh update from the master as we transition from a start of not having a master to having one
|
||||
|
@ -381,7 +398,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
// we don't want to notify
|
||||
if (newClusterState.nodes().localNodeMaster()) {
|
||||
logger.debug("publishing cluster state version {}", newClusterState.version());
|
||||
discoveryService.publish(newClusterState);
|
||||
discoveryService.publish(newClusterState, ackListener);
|
||||
}
|
||||
|
||||
// update the current cluster state
|
||||
|
@ -409,18 +426,26 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
});
|
||||
}
|
||||
|
||||
//manual ack only from the master at the end of the publish
|
||||
if (newClusterState.nodes().localNodeMaster()) {
|
||||
try {
|
||||
ackListener.onNodeAck(localNode(), null);
|
||||
} catch(Throwable t) {
|
||||
logger.debug("error while processing ack for master node [{}]", t, newClusterState.nodes().localNode());
|
||||
}
|
||||
}
|
||||
|
||||
if (updateTask instanceof ProcessedClusterStateUpdateTask) {
|
||||
((ProcessedClusterStateUpdateTask) updateTask).clusterStateProcessed(source, previousClusterState, newClusterState);
|
||||
}
|
||||
|
||||
logger.debug("processing [{}]: done applying updated cluster_state (version: {})", source, newClusterState.version());
|
||||
} catch (Exception e) {
|
||||
} catch (Throwable t) {
|
||||
StringBuilder sb = new StringBuilder("failed to apply updated cluster state:\nversion [").append(newClusterState.version()).append("], source [").append(source).append("]\n");
|
||||
sb.append(newClusterState.nodes().prettyPrint());
|
||||
sb.append(newClusterState.routingTable().prettyPrint());
|
||||
sb.append(newClusterState.readOnlyRoutingNodes().prettyPrint());
|
||||
logger.warn(sb.toString(), e);
|
||||
logger.warn(sb.toString(), t);
|
||||
// TODO: do we want to call updateTask.onFailure here?
|
||||
}
|
||||
}
|
||||
|
@ -584,4 +609,73 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
listener.offMaster();
|
||||
}
|
||||
}
|
||||
|
||||
private static class NoOpAckListener implements Discovery.AckListener {
|
||||
@Override
|
||||
public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimeout() {
|
||||
}
|
||||
}
|
||||
|
||||
private class AckCountDownListener implements Discovery.AckListener {
|
||||
private final AckedClusterStateUpdateTask ackedUpdateTask;
|
||||
private final long version;
|
||||
private final AtomicInteger countDown;
|
||||
private final AtomicBoolean notified = new AtomicBoolean(false);
|
||||
private final Future<?> ackTimeoutCallback;
|
||||
private Throwable lastFailure;
|
||||
|
||||
AckCountDownListener(AckedClusterStateUpdateTask ackedUpdateTask, long clusterStateVersion, DiscoveryNodes nodes, ThreadPool threadPool) {
|
||||
this.ackedUpdateTask = ackedUpdateTask;
|
||||
this.version = clusterStateVersion;
|
||||
int countDown = 0;
|
||||
for (DiscoveryNode node : nodes) {
|
||||
if (ackedUpdateTask.mustAck(node)) {
|
||||
countDown++;
|
||||
}
|
||||
}
|
||||
logger.trace("expecting {} acknowledgements for cluster_state update (version: {})", countDown, version);
|
||||
this.countDown = new AtomicInteger(countDown);
|
||||
this.ackTimeoutCallback = threadPool.schedule(ackedUpdateTask.ackTimeout(), ThreadPool.Names.GENERIC, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
onTimeout();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) {
|
||||
if (!ackedUpdateTask.mustAck(node)) {
|
||||
return;
|
||||
}
|
||||
if (t == null) {
|
||||
logger.trace("ack received from node [{}], cluster_state update (version: {})", node, version);
|
||||
} else {
|
||||
this.lastFailure = t;
|
||||
logger.debug("ack received from node [{}], cluster_state update (version: {})", t, node, version);
|
||||
}
|
||||
|
||||
assert countDown.get() > 0;
|
||||
if (countDown.decrementAndGet() == 0) {
|
||||
if (notified.compareAndSet(false, true) ) {
|
||||
logger.trace("all expected nodes acknowledged cluster_state update (version: {})", version);
|
||||
ackTimeoutCallback.cancel(true);
|
||||
ackedUpdateTask.onAllNodesAcked(lastFailure);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimeout() {
|
||||
if (notified.compareAndSet(false, true)) {
|
||||
logger.trace("timeout waiting for acknowledgement for cluster_state update (version: {})", version);
|
||||
ackedUpdateTask.onAckTimeout();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* 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.discovery;
|
||||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
|
||||
/**
|
||||
* Allows to wait for all nodes to reply to the publish of a new cluster state
|
||||
* and notifies the {@link org.elasticsearch.discovery.Discovery.AckListener}
|
||||
* so that the cluster state update can be acknowledged
|
||||
*/
|
||||
public class AckClusterStatePublishResponseHandler extends BlockingClusterStatePublishResponseHandler {
|
||||
|
||||
private static final ESLogger logger = ESLoggerFactory.getLogger(AckClusterStatePublishResponseHandler.class.getName());
|
||||
|
||||
private final Discovery.AckListener ackListener;
|
||||
|
||||
/**
|
||||
* Creates a new AckClusterStatePublishResponseHandler
|
||||
* @param nonMasterNodes number of nodes that are supposed to reply to a cluster state publish from master
|
||||
* @param ackListener the {@link org.elasticsearch.discovery.Discovery.AckListener} to notify for each response
|
||||
* gotten from non master nodes
|
||||
*/
|
||||
public AckClusterStatePublishResponseHandler(int nonMasterNodes, Discovery.AckListener ackListener) {
|
||||
//Don't count the master as acknowledged, because it's not done yet
|
||||
//otherwise we might end up with all the nodes but the master holding the latest cluster state
|
||||
super(nonMasterNodes);
|
||||
this.ackListener = ackListener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponse(DiscoveryNode node) {
|
||||
super.onResponse(node);
|
||||
onNodeAck(ackListener, node, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(DiscoveryNode node, Throwable t) {
|
||||
try {
|
||||
super.onFailure(node, t);
|
||||
} finally {
|
||||
onNodeAck(ackListener, node, t);
|
||||
}
|
||||
}
|
||||
|
||||
private void onNodeAck(final Discovery.AckListener ackListener, DiscoveryNode node, Throwable t) {
|
||||
try {
|
||||
ackListener.onNodeAck(node, t);
|
||||
} catch (Throwable t1) {
|
||||
logger.debug("error while processing ack for node [{}]", t1, node);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* 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.discovery;
|
||||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
||||
/**
|
||||
* Default implementation of {@link ClusterStatePublishResponseHandler}, allows to await a reply
|
||||
* to a cluster state publish from all non master nodes, up to a timeout
|
||||
*/
|
||||
public class BlockingClusterStatePublishResponseHandler implements ClusterStatePublishResponseHandler {
|
||||
|
||||
private final CountDownLatch latch;
|
||||
|
||||
/**
|
||||
* Creates a new BlockingClusterStatePublishResponseHandler
|
||||
* @param nonMasterNodes number of nodes that are supposed to reply to a cluster state publish from master
|
||||
*/
|
||||
public BlockingClusterStatePublishResponseHandler(int nonMasterNodes) {
|
||||
//Don't count the master, as it's the one that does the publish
|
||||
//the master won't call onResponse either
|
||||
this.latch = new CountDownLatch(nonMasterNodes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponse(DiscoveryNode node) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(DiscoveryNode node, Throwable t) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean awaitAllNodes(TimeValue timeout) throws InterruptedException {
|
||||
return latch.await(timeout.millis(), TimeUnit.MILLISECONDS);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* 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.discovery;
|
||||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
/**
|
||||
* Handles responses obtained when publishing a new cluster state from master to all non master nodes.
|
||||
* Allows to await a reply from all non master nodes, up to a timeout
|
||||
*/
|
||||
public interface ClusterStatePublishResponseHandler {
|
||||
|
||||
/**
|
||||
* Called for each response obtained from non master nodes
|
||||
* @param node the node that replied to the publish event
|
||||
*/
|
||||
void onResponse(DiscoveryNode node);
|
||||
|
||||
/**
|
||||
* Called for each failure obtained from non master nodes
|
||||
* @param node the node that replied to the publish event
|
||||
*/
|
||||
void onFailure(DiscoveryNode node, Throwable t);
|
||||
|
||||
/**
|
||||
* Allows to wait for all non master nodes to reply to the publish event up to a timeout
|
||||
* @param timeout the timeout
|
||||
* @return true if the timeout expired or not, false otherwise
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
boolean awaitAllNodes(TimeValue timeout) throws InterruptedException;
|
||||
}
|
|
@ -24,8 +24,8 @@ import org.elasticsearch.cluster.block.ClusterBlock;
|
|||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.component.LifecycleComponent;
|
||||
import org.elasticsearch.common.inject.internal.Nullable;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
|
@ -60,6 +60,14 @@ public interface Discovery extends LifecycleComponent<Discovery> {
|
|||
/**
|
||||
* Publish all the changes to the cluster from the master (can be called just by the master). The publish
|
||||
* process should not publish this state to the master as well! (the master is sending it...).
|
||||
*
|
||||
* The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether
|
||||
* they updated their own cluster state or not.
|
||||
*/
|
||||
void publish(ClusterState clusterState);
|
||||
void publish(ClusterState clusterState, AckListener ackListener);
|
||||
|
||||
public static interface AckListener {
|
||||
void onNodeAck(DiscoveryNode node, @Nullable Throwable t);
|
||||
void onTimeout();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -107,11 +107,13 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
|
|||
/**
|
||||
* Publish all the changes to the cluster from the master (can be called just by the master). The publish
|
||||
* process should not publish this state to the master as well! (the master is sending it...).
|
||||
*
|
||||
* The {@link org.elasticsearch.discovery.Discovery.AckListener} allows to acknowledge the publish
|
||||
* event based on the response gotten from all nodes
|
||||
*/
|
||||
public void publish(ClusterState clusterState) {
|
||||
if (!lifecycle.started()) {
|
||||
return;
|
||||
public void publish(ClusterState clusterState, Discovery.AckListener ackListener) {
|
||||
if (lifecycle.started()) {
|
||||
discovery.publish(clusterState, ackListener);
|
||||
}
|
||||
discovery.publish(clusterState);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,8 +35,7 @@ import org.elasticsearch.common.inject.internal.Nullable;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.InitialStateDiscoveryListener;
|
||||
import org.elasticsearch.discovery.*;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
|
@ -44,8 +43,6 @@ import java.util.Queue;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
|
@ -58,6 +55,8 @@ import static org.elasticsearch.cluster.ClusterState.newClusterStateBuilder;
|
|||
*/
|
||||
public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery {
|
||||
|
||||
private static final LocalDiscovery[] NO_MEMBERS = new LocalDiscovery[0];
|
||||
|
||||
private final TransportService transportService;
|
||||
private final ClusterService clusterService;
|
||||
private final DiscoveryNodeService discoveryNodeService;
|
||||
|
@ -277,24 +276,33 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
|
|||
return clusterName.value() + "/" + localNode.id();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publish(ClusterState clusterState) {
|
||||
public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) {
|
||||
if (!master) {
|
||||
throw new ElasticSearchIllegalStateException("Shouldn't publish state when not master");
|
||||
}
|
||||
LocalDiscovery[] members = members();
|
||||
if (members.length > 0) {
|
||||
publish(members, clusterState, new AckClusterStatePublishResponseHandler(members.length - 1, ackListener));
|
||||
}
|
||||
}
|
||||
|
||||
private LocalDiscovery[] members() {
|
||||
ClusterGroup clusterGroup = clusterGroups.get(clusterName);
|
||||
if (clusterGroup == null) {
|
||||
// nothing to publish to
|
||||
return;
|
||||
return NO_MEMBERS;
|
||||
}
|
||||
Queue<LocalDiscovery> members = clusterGroup.members();
|
||||
return members.toArray(new LocalDiscovery[members.size()]);
|
||||
}
|
||||
|
||||
private void publish(LocalDiscovery[] members, ClusterState clusterState, final ClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
|
||||
try {
|
||||
// we do the marshaling intentionally, to check it works well...
|
||||
final byte[] clusterStateBytes = Builder.toBytes(clusterState);
|
||||
LocalDiscovery[] members = clusterGroup.members().toArray(new LocalDiscovery[0]);
|
||||
final CountDownLatch latch = new CountDownLatch(members.length);
|
||||
for (LocalDiscovery discovery : members) {
|
||||
|
||||
for (final LocalDiscovery discovery : members) {
|
||||
if (discovery.master) {
|
||||
latch.countDown();
|
||||
continue;
|
||||
}
|
||||
final ClusterState nodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode);
|
||||
|
@ -318,23 +326,23 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
|
|||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
logger.error("unexpected failure during [{}]", t, source);
|
||||
latch.countDown();
|
||||
publishResponseHandler.onFailure(discovery.localNode, t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
sendInitialStateEventIfNeeded();
|
||||
latch.countDown();
|
||||
publishResponseHandler.onResponse(discovery.localNode);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
latch.countDown();
|
||||
publishResponseHandler.onResponse(discovery.localNode);
|
||||
}
|
||||
}
|
||||
|
||||
if (publishTimeout.millis() > 0) {
|
||||
try {
|
||||
boolean awaited = latch.await(publishTimeout.millis(), TimeUnit.MILLISECONDS);
|
||||
boolean awaited = publishResponseHandler.awaitAllNodes(publishTimeout);
|
||||
if (!awaited) {
|
||||
logger.debug("awaiting all nodes to process published state {} timed out, timeout {}", clusterState.version(), publishTimeout);
|
||||
}
|
||||
|
|
|
@ -262,13 +262,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
}
|
||||
|
||||
@Override
|
||||
public void publish(ClusterState clusterState) {
|
||||
public void publish(ClusterState clusterState, AckListener ackListener) {
|
||||
if (!master) {
|
||||
throw new ElasticSearchIllegalStateException("Shouldn't publish state when not master");
|
||||
}
|
||||
latestDiscoNodes = clusterState.nodes();
|
||||
nodesFD.updateNodes(clusterState.nodes());
|
||||
publishClusterState.publish(clusterState);
|
||||
publishClusterState.publish(clusterState, ackListener);
|
||||
}
|
||||
|
||||
private void asyncJoinCluster() {
|
||||
|
|
|
@ -30,14 +30,15 @@ import org.elasticsearch.common.compress.CompressorFactory;
|
|||
import org.elasticsearch.common.io.stream.*;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.discovery.AckClusterStatePublishResponseHandler;
|
||||
import org.elasticsearch.discovery.ClusterStatePublishResponseHandler;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -78,16 +79,18 @@ public class PublishClusterStateAction extends AbstractComponent {
|
|||
transportService.removeHandler(PublishClusterStateRequestHandler.ACTION);
|
||||
}
|
||||
|
||||
public void publish(ClusterState clusterState) {
|
||||
public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) {
|
||||
publish(clusterState, new AckClusterStatePublishResponseHandler(clusterState.nodes().size()-1, ackListener));
|
||||
}
|
||||
|
||||
private void publish(ClusterState clusterState, final ClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
|
||||
DiscoveryNode localNode = nodesProvider.nodes().localNode();
|
||||
|
||||
Map<Version, BytesReference> serializedStates = Maps.newHashMap();
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(clusterState.nodes().size());
|
||||
for (final DiscoveryNode node : clusterState.nodes()) {
|
||||
if (node.equals(localNode)) {
|
||||
// no need to send to our self
|
||||
latch.countDown();
|
||||
continue;
|
||||
}
|
||||
// try and serialize the cluster state once (or per version), so we don't serialize it
|
||||
|
@ -104,7 +107,7 @@ public class PublishClusterStateAction extends AbstractComponent {
|
|||
serializedStates.put(node.version(), bytes);
|
||||
} catch (Throwable e) {
|
||||
logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node);
|
||||
latch.countDown();
|
||||
publishResponseHandler.onFailure(node, e);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
@ -120,24 +123,25 @@ public class PublishClusterStateAction extends AbstractComponent {
|
|||
|
||||
@Override
|
||||
public void handleResponse(TransportResponse.Empty response) {
|
||||
latch.countDown();
|
||||
publishResponseHandler.onResponse(node);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleException(TransportException exp) {
|
||||
logger.debug("failed to send cluster state to [{}]", exp, node);
|
||||
latch.countDown();
|
||||
publishResponseHandler.onFailure(node, exp);
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
latch.countDown();
|
||||
logger.debug("error sending cluster state to [{}]", t, node);
|
||||
publishResponseHandler.onFailure(node, t);
|
||||
}
|
||||
}
|
||||
|
||||
if (publishTimeout.millis() > 0) {
|
||||
// only wait if the publish timeout is configured...
|
||||
try {
|
||||
boolean awaited = latch.await(publishTimeout.millis(), TimeUnit.MILLISECONDS);
|
||||
boolean awaited = publishResponseHandler.awaitAllNodes(publishTimeout);
|
||||
if (!awaited) {
|
||||
logger.debug("awaiting all nodes to process published state {} timed out, timeout {}", clusterState.version(), publishTimeout);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue