Author: goffinet Date: Mon May 30 08:51:33 2011 New Revision: 1129065 URL: http://svn.apache.org/viewvc?rev=1129065&view=rev Log: Committed second patch for making bootstrap retry. Added a default bootstrap timeout. patch by goffinet; reviewed by stuhood for CASSANDRA-2644
Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=1129065&r1=1129064&r2=1129065&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java Mon May 30 08:51:33 2011 @@ -22,6 +22,7 @@ package org.apache.cassandra.dht; import java.net.InetAddress; import java.util.*; import java.util.concurrent.CountDownLatch; + import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import com.google.common.base.Charsets; @@ -61,6 +62,7 @@ public class BootStrapper /* tokens of the nodes being bootstrapped. */ protected final Token token; protected final TokenMetadata tokenMetadata; + private static final long BOOTSTRAP_TIMEOUT = 30000; // default bootstrap timeout of 30s public BootStrapper(InetAddress address, Token token, TokenMetadata tmd) { @@ -220,9 +222,20 @@ public class BootStrapper StorageService.Verb.BOOTSTRAP_TOKEN, ArrayUtils.EMPTY_BYTE_ARRAY, Gossiper.instance.getVersion(maxEndpoint)); - BootstrapTokenCallback btc = new BootstrapTokenCallback(); - MessagingService.instance().sendRR(message, maxEndpoint, btc); - return btc.getToken(); + int retries = 5; + long timeout = Math.max(MessagingService.getDefaultCallbackTimeout(), BOOTSTRAP_TIMEOUT); + + while (retries > 0) + { + BootstrapTokenCallback btc = new BootstrapTokenCallback(); + MessagingService.instance().sendRR(message, maxEndpoint, btc, timeout); + Token token = btc.getToken(timeout); + if (token != null) + return token; + + retries--; + } + throw new RuntimeException("Bootstrap failed, could not obtain token from: " + maxEndpoint); } public static Multimap<InetAddress, Range> getWorkMap(Multimap<Range, InetAddress> rangesWithSourceTarget) @@ -269,17 +282,19 @@ public class BootStrapper private volatile Token<?> token; private final Condition condition = new SimpleCondition(); - public Token<?> getToken() + public Token<?> getToken(long timeout) { + boolean success; try { - condition.await(); + success = condition.await(timeout, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { throw new RuntimeException(e); } - return token; + + return success ? token : null; } public void response(Message msg)