Skip to content

Commit

Permalink
Discovery: publishing timeout to log at WARN and indicate pending nodes
Browse files Browse the repository at this point in the history
When the master publishes a new cluster state it waits (by default) for up to 30s for all nodes to respond. If not it continues to process other pending tasks. At the moment, this timeout is logged under DEBUG but it typically represent a serious issue with one or more of the nodes. We should log it in WARN and give the nodes that failed to respond in a timefly fashion

Closes #9551
  • Loading branch information
bleskes committed Feb 4, 2015
1 parent ebd0f93 commit c5312ff
Show file tree
Hide file tree
Showing 6 changed files with 198 additions and 77 deletions.
Expand Up @@ -22,6 +22,8 @@
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.ESLoggerFactory;

import java.util.Set;

/**
* 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}
Expand All @@ -35,14 +37,14 @@ public class AckClusterStatePublishResponseHandler extends BlockingClusterStateP

/**
* Creates a new AckClusterStatePublishResponseHandler
* @param nonMasterNodes number of nodes that are supposed to reply to a cluster state publish from master
* @param publishingToNodes the set of nodes to which the cluster state will be published and should respond
* @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) {
public AckClusterStatePublishResponseHandler(Set<DiscoveryNode> publishingToNodes, 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);
super(publishingToNodes);
this.ackListener = ackListener;
}

Expand Down
Expand Up @@ -20,41 +20,71 @@

import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;

import java.util.Set;
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
* 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 class BlockingClusterStatePublishResponseHandler implements ClusterStatePublishResponseHandler {
public class BlockingClusterStatePublishResponseHandler {

private final CountDownLatch latch;
private final Set<DiscoveryNode> pendingNodes;

/**
* Creates a new BlockingClusterStatePublishResponseHandler
* @param nonMasterNodes number of nodes that are supposed to reply to a cluster state publish from master
* @param publishingToNodes the set of nodes to which the cluster state will be published and should respond
*/
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);
public BlockingClusterStatePublishResponseHandler(Set<DiscoveryNode> publishingToNodes) {
this.pendingNodes = ConcurrentCollections.newConcurrentSet();
this.pendingNodes.addAll(publishingToNodes);
this.latch = new CountDownLatch(pendingNodes.size());
}

@Override
/**
* Called for each response obtained from non master nodes
*
* @param node the node that replied to the publish event
*/
public void onResponse(DiscoveryNode node) {
boolean found = pendingNodes.remove(node);
assert found : "node [" + node + "] already responded or failed";
latch.countDown();
}

@Override
/**
* Called for each failure obtained from non master nodes
* @param node the node that replied to the publish event
*/
public void onFailure(DiscoveryNode node, Throwable t) {
boolean found = pendingNodes.remove(node);
assert found : "node [" + node + "] already responded or failed";
latch.countDown();
}

@Override
/**
* 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
*/
public boolean awaitAllNodes(TimeValue timeout) throws InterruptedException {
return latch.await(timeout.millis(), TimeUnit.MILLISECONDS);
boolean success = latch.await(timeout.millis(), TimeUnit.MILLISECONDS);
assert !success || pendingNodes.isEmpty() : "response count reached 0 but still waiting for some nodes";
return success;
}

/**
* returns a list of nodes which didn't respond yet
*/
public DiscoveryNode[] pendingNodes() {
// we use a zero length array, because if we try to pre allocate we may need to remove trailing
// nulls if some nodes responded in the meanwhile
return pendingNodes.toArray(new DiscoveryNode[0]);
}
}

This file was deleted.

Expand Up @@ -40,6 +40,7 @@
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.transport.TransportService;

import java.util.HashSet;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
Expand Down Expand Up @@ -278,7 +279,14 @@ public void publish(ClusterState clusterState, final Discovery.AckListener ackLi
}
LocalDiscovery[] members = members();
if (members.length > 0) {
publish(members, clusterState, new AckClusterStatePublishResponseHandler(members.length - 1, ackListener));
Set<DiscoveryNode> nodesToPublishTo = new HashSet<>(members.length);
for (LocalDiscovery localDiscovery : members) {
if (localDiscovery.master) {
continue;
}
nodesToPublishTo.add(localDiscovery.localNode);
}
publish(members, clusterState, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener));
}
}

Expand All @@ -291,7 +299,7 @@ private LocalDiscovery[] members() {
return members.toArray(new LocalDiscovery[members.size()]);
}

private void publish(LocalDiscovery[] members, ClusterState clusterState, final ClusterStatePublishResponseHandler publishResponseHandler) {
private void publish(LocalDiscovery[] members, ClusterState clusterState, final BlockingClusterStatePublishResponseHandler publishResponseHandler) {

try {
// we do the marshaling intentionally, to check it works well...
Expand Down Expand Up @@ -355,7 +363,11 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS
try {
boolean awaited = publishResponseHandler.awaitAllNodes(publishTimeout);
if (!awaited) {
logger.debug("awaiting all nodes to process published state {} timed out, timeout {}", clusterState.version(), publishTimeout);
DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
// everyone may have just responded
if (pendingNodes.length > 0) {
logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", clusterState.version(), publishTimeout, pendingNodes);
}
}
} catch (InterruptedException e) {
// ignore & restore interrupt
Expand Down
Expand Up @@ -32,14 +32,16 @@
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.BlockingClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;

/**
Expand Down Expand Up @@ -83,22 +85,27 @@ public void close() {
}

public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) {
publish(clusterState, new AckClusterStatePublishResponseHandler(clusterState.nodes().size() - 1, ackListener));
Set<DiscoveryNode> nodesToPublishTo = new HashSet<>(clusterState.nodes().size());
DiscoveryNode localNode = nodesProvider.nodes().localNode();
for (final DiscoveryNode node : clusterState.nodes()) {
if (node.equals(localNode)) {
continue;
}
nodesToPublishTo.add(node);
}
publish(clusterState, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener));
}

private void publish(final ClusterState clusterState, final ClusterStatePublishResponseHandler publishResponseHandler) {

DiscoveryNode localNode = nodesProvider.nodes().localNode();
private void publish(final ClusterState clusterState, final Set<DiscoveryNode> nodesToPublishTo,
final BlockingClusterStatePublishResponseHandler publishResponseHandler) {

Map<Version, BytesReference> serializedStates = Maps.newHashMap();

final AtomicBoolean timedOutWaitingForNodes = new AtomicBoolean(false);
final TimeValue publishTimeout = discoverySettings.getPublishTimeout();

for (final DiscoveryNode node : clusterState.nodes()) {
if (node.equals(localNode)) {
continue;
}
for (final DiscoveryNode node : nodesToPublishTo) {

// try and serialize the cluster state once (or per version), so we don't serialize it
// per node when we send it over the wire, compress it while we are at it...
BytesReference bytes = serializedStates.get(node.version());
Expand Down Expand Up @@ -152,7 +159,11 @@ public void handleException(TransportException exp) {
try {
timedOutWaitingForNodes.set(!publishResponseHandler.awaitAllNodes(publishTimeout));
if (timedOutWaitingForNodes.get()) {
logger.debug("timed out waiting for all nodes to process published state [{}] (timeout [{}])", clusterState.version(), publishTimeout);
DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
// everyone may have just responded
if (pendingNodes.length > 0) {
logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", clusterState.version(), publishTimeout, pendingNodes);
}
}
} catch (InterruptedException e) {
// ignore & restore interrupt
Expand Down
@@ -0,0 +1,115 @@
/*
* Licensed to Elasticsearch 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.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.test.ElasticsearchTestCase;

import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.CyclicBarrier;

import static org.hamcrest.Matchers.*;

public class BlockingClusterStatePublishResponseHandlerTests extends ElasticsearchTestCase {

static private class PublishResponder extends AbstractRunnable {

final boolean fail;
final DiscoveryNode node;
final CyclicBarrier barrier;
final ESLogger logger;
final BlockingClusterStatePublishResponseHandler handler;

public PublishResponder(boolean fail, DiscoveryNode node, CyclicBarrier barrier, ESLogger logger, BlockingClusterStatePublishResponseHandler handler) {
this.fail = fail;

this.node = node;
this.barrier = barrier;
this.logger = logger;
this.handler = handler;
}

@Override
public void onFailure(Throwable t) {
logger.error("unexpected error", t);
}

@Override
protected void doRun() throws Exception {
barrier.await();
if (fail) {
handler.onFailure(node, new Exception("bla"));
} else {
handler.onResponse(node);
}
}
}

public void testConcurrentAccess() throws InterruptedException {
int nodeCount = scaledRandomIntBetween(10, 20);
DiscoveryNode[] allNodes = new DiscoveryNode[nodeCount];
for (int i = 0; i < nodeCount; i++) {
DiscoveryNode node = new DiscoveryNode("node_" + i, DummyTransportAddress.INSTANCE, Version.CURRENT);
allNodes[i] = node;
}

BlockingClusterStatePublishResponseHandler handler = new BlockingClusterStatePublishResponseHandler(new HashSet<>(Arrays.asList(allNodes)));

int firstRound = randomIntBetween(5, nodeCount - 1);
Thread[] threads = new Thread[firstRound];
CyclicBarrier barrier = new CyclicBarrier(firstRound);
Set<DiscoveryNode> completedNodes = new HashSet<>();
for (int i = 0; i < threads.length; i++) {
completedNodes.add(allNodes[i]);
threads[i] = new Thread(new PublishResponder(randomBoolean(), allNodes[i], barrier, logger, handler));
threads[i].start();
}
// wait on the threads to finish
for (Thread t : threads) {
t.join();
}
// verify that the publisher times out
assertFalse("expected handler wait to timeout as not all nodes responded", handler.awaitAllNodes(new TimeValue(10)));
Set<DiscoveryNode> pendingNodes = new HashSet<>(Arrays.asList(handler.pendingNodes()));
assertThat(completedNodes, not(contains(pendingNodes.toArray(new DiscoveryNode[0]))));
assertThat(completedNodes.size() + pendingNodes.size(), equalTo(allNodes.length));
int secondRound = allNodes.length - firstRound;
threads = new Thread[secondRound];
barrier = new CyclicBarrier(secondRound);

for (int i = 0; i < threads.length; i++) {
threads[i] = new Thread(new PublishResponder(randomBoolean(), allNodes[firstRound + i], barrier, logger, handler));
threads[i].start();
}
// wait on the threads to finish
for (Thread t : threads) {
t.join();
}
assertTrue("expected handler not to timeout as all nodes responded", handler.awaitAllNodes(new TimeValue(10)));
assertThat(handler.pendingNodes(), arrayWithSize(0));

}
}

0 comments on commit c5312ff

Please sign in to comment.