GOSSIP-22 New failure detector

This commit is contained in:
Edward Capriolo
2017-01-06 20:59:32 -05:00
parent 18c9f911d1
commit f56d282d30
16 changed files with 428 additions and 235 deletions

View File

@ -27,16 +27,13 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import javax.management.Notification;
import javax.management.NotificationListener;
import org.apache.log4j.Logger;
import org.apache.gossip.GossipMember;
import org.apache.gossip.GossipService;
import org.apache.gossip.GossipSettings;
import org.apache.gossip.LocalGossipMember;
import org.apache.gossip.event.GossipListener;
@ -47,7 +44,7 @@ import org.apache.gossip.model.GossipDataMessage;
import org.apache.gossip.model.SharedGossipDataMessage;
public abstract class GossipManager implements NotificationListener {
public abstract class GossipManager {
public static final Logger LOGGER = Logger.getLogger(GossipManager.class);
@ -74,6 +71,8 @@ public abstract class GossipManager implements NotificationListener {
private final DataReaper dataReaper;
private final Clock clock;
private final ScheduledExecutorService scheduledServiced;
public GossipManager(String cluster,
URI uri, String id, GossipSettings settings,
@ -83,60 +82,26 @@ public abstract class GossipManager implements NotificationListener {
gossipCore = new GossipCore(this);
clock = new SystemClock();
dataReaper = new DataReaper(gossipCore, clock);
me = new LocalGossipMember(cluster, uri, id, System.currentTimeMillis(), this,
settings.getCleanupInterval());
me = new LocalGossipMember(cluster, uri, id, clock.nanoTime(),
+ settings.getWindowSize(), settings.getMinimumSamples());
members = new ConcurrentSkipListMap<>();
for (GossipMember startupMember : gossipMembers) {
if (!startupMember.equals(me)) {
LocalGossipMember member = new LocalGossipMember(startupMember.getClusterName(),
startupMember.getUri(), startupMember.getId(),
System.currentTimeMillis(), this, settings.getCleanupInterval());
members.put(member, GossipState.UP);
GossipService.LOGGER.debug(member);
clock.nanoTime(), settings.getWindowSize(), settings.getMinimumSamples());
//TODO should members start in down state?
members.put(member, GossipState.DOWN);
}
}
gossipThreadExecutor = Executors.newCachedThreadPool();
gossipServiceRunning = new AtomicBoolean(true);
this.listener = listener;
Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
public void run() {
GossipService.LOGGER.debug("Service has been shutdown...");
}
}));
this.scheduledServiced = Executors.newScheduledThreadPool(1);
}
/**
* All timers associated with a member will trigger this method when it goes off. The timer will
* go off if we have not heard from this member in <code> _settings.T_CLEANUP </code> time.
*/
@Override
public void handleNotification(Notification notification, Object handback) {
LocalGossipMember deadMember = (LocalGossipMember) notification.getUserData();
GossipService.LOGGER.debug("Dead member detected: " + deadMember);
members.put(deadMember, GossipState.DOWN);
if (listener != null) {
listener.gossipEvent(deadMember, GossipState.DOWN);
}
}
public void reviveMember(LocalGossipMember m) {
for (Entry<LocalGossipMember, GossipState> it : this.members.entrySet()) {
if (it.getKey().getId().equals(m.getId())) {
it.getKey().disableTimer();
}
}
members.remove(m);
members.put(m, GossipState.UP);
if (listener != null) {
listener.gossipEvent(m, GossipState.UP);
}
}
public void createOrReviveMember(LocalGossipMember m) {
members.put(m, GossipState.UP);
if (listener != null) {
listener.gossipEvent(m, GossipState.UP);
}
public ConcurrentSkipListMap<LocalGossipMember, GossipState> getMembers() {
return members;
}
public GossipSettings getSettings() {
@ -181,17 +146,44 @@ public abstract class GossipManager implements NotificationListener {
* thread and start the receiver thread.
*/
public void init() {
for (LocalGossipMember member : members.keySet()) {
if (member != me) {
member.startTimeoutTimer();
}
}
passiveGossipThread = new OnlyProcessReceivedPassiveGossipThread(this, gossipCore);
gossipThreadExecutor.execute(passiveGossipThread);
activeGossipThread = new ActiveGossipThread(this, this.gossipCore);
activeGossipThread.init();
dataReaper.init();
GossipService.LOGGER.debug("The GossipService is started.");
scheduledServiced.scheduleAtFixedRate(() -> {
try {
for (Entry<LocalGossipMember, GossipState> entry : members.entrySet()) {
Double result = null;
try {
result = entry.getKey().detect(clock.nanoTime());
//System.out.println(entry.getKey() +" "+ result);
if (result != null) {
if (result > settings.getConvictThreshold() && entry.getValue() == GossipState.UP) {
members.put(entry.getKey(), GossipState.DOWN);
listener.gossipEvent(entry.getKey(), GossipState.DOWN);
}
if (result <= settings.getConvictThreshold() && entry.getValue() == GossipState.DOWN) {
members.put(entry.getKey(), GossipState.UP);
listener.gossipEvent(entry.getKey(), GossipState.UP);
}
}
} catch (IllegalArgumentException ex) {
//0.0 returns throws exception computing the mean.
long now = clock.nanoTime();
long nowInMillis = TimeUnit.MILLISECONDS.convert(now,TimeUnit.NANOSECONDS);
if (nowInMillis - settings.getCleanupInterval() > entry.getKey().getHeartbeat() && entry.getValue() == GossipState.UP){
LOGGER.warn("Marking down");
members.put(entry.getKey(), GossipState.DOWN);
listener.gossipEvent(entry.getKey(), GossipState.DOWN);
}
} //end catch
} // end for
} catch (RuntimeException ex) {
LOGGER.warn("scheduled state had exception", ex);
}
}, 0, 100, TimeUnit.MILLISECONDS);
LOGGER.debug("The GossipManager is started.");
}
/**