Skip to content

Commit

Permalink
Added generic cluster state update ack mechanism
Browse files Browse the repository at this point in the history
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
  • Loading branch information
javanna committed Oct 15, 2013
1 parent f5e2cf9 commit 55f1eab
Show file tree
Hide file tree
Showing 10 changed files with 389 additions and 38 deletions.
Original file line number Diff line number Diff line change
@@ -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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
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;
Expand All @@ -49,6 +50,8 @@
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;
Expand Down Expand Up @@ -317,6 +320,7 @@ public void run() {
}

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);
Expand All @@ -327,6 +331,19 @@ public void run() {
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
Expand Down Expand Up @@ -381,7 +398,7 @@ public void run() {
// 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
Expand Down Expand Up @@ -409,18 +426,26 @@ public void run() {
});
}

//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?
}
}
Expand Down Expand Up @@ -584,4 +609,73 @@ public void run() {
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();
}
}
}

}
Original file line number Diff line number Diff line change
@@ -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);
}
}
}
Original file line number Diff line number Diff line change
@@ -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);
}
}
Original file line number Diff line number Diff line change
@@ -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;
}

0 comments on commit 55f1eab

Please sign in to comment.