@Override public void isLeader() { callback.leaderCallBack(); LOG.info("{} gained leadership.", hostName); }
@Override public void notLeader() { LOG.info("{} lost leadership.", hostName); //Just to be sure callback.notLeaderCallback(); } };
numGainedLeader.mark(); setUpNimbusInfo(acls); ClusterUtils.STORMS_SUBTREE, false)); Set<String> activeTopologyBlobKeys = populateTopologyBlobKeys(activeTopologyIds); Set<String> activeTopologyCodeKeys = filterTopologyCodeKeys(activeTopologyBlobKeys); Set<String> allLocalBlobKeys = Sets.newHashSet(blobStore.listKeys()); Set<String> allLocalTopologyBlobKeys = filterTopologyBlobKeys(allLocalBlobKeys); generateJoinedString(activeTopologyIds), generateJoinedString(allLocalTopologyBlobKeys), generateJoinedString(diffTopology)); Set<String> activeTopologyDependencies = getTopologyDependencyKeys(activeTopologyCodeKeys); generateJoinedString(activeTopologyDependencies), generateJoinedString(allLocalBlobKeys), generateJoinedString(diffDependencies)); LOG.info("Code for all active topologies is available locally, but some dependencies are not found locally, " + "giving up leadership."); closeLatch(); closeLatch();
protected ILeaderElector zkLeaderElectorImpl(Map<String, Object> conf, CuratorFramework zk, BlobStore blobStore, final TopoCache tc, IStormClusterState clusterState, List<ACL> acls, StormMetricsRegistry metricsRegistry) throws UnknownHostException { List<String> servers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS); String leaderLockPath = "/leader-lock"; String id = NimbusInfo.fromConf(conf).toHostPortString(); AtomicReference<LeaderLatch> leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id)); AtomicReference<LeaderLatchListener> leaderLatchListenerAtomicReference = new AtomicReference<>(leaderLatchListenerImpl( new LeaderListenerCallback(conf, zk, leaderLatchAtomicReference.get(), blobStore, tc, clusterState, acls, metricsRegistry))); return new LeaderElectorImp(conf, servers, zk, leaderLockPath, id, leaderLatchAtomicReference, leaderLatchListenerAtomicReference, blobStore, tc, clusterState, acls, metricsRegistry); }
@Override public void addToLeaderLockQueue() throws Exception { // if this latch is already closed, we need to create new instance. if (LeaderLatch.State.CLOSED.equals(leaderLatch.get().getState())) { leaderLatch.set(new LeaderLatch(zk, leaderlockPath)); LeaderListenerCallback callback = new LeaderListenerCallback(conf, zk, leaderLatch.get(), blobStore, tc, clusterState, acls, metricsRegistry); leaderLatchListener.set(Zookeeper.leaderLatchListenerImpl(callback)); LOG.info("LeaderLatch was in closed state. Resetted the leaderLatch and listeners."); } // Only if the latch is not already started we invoke start if (LeaderLatch.State.LATENT.equals(leaderLatch.get().getState())) { leaderLatch.get().addListener(leaderLatchListener.get()); leaderLatch.get().start(); LOG.info("Queued up for leader lock."); } else { LOG.info("Node already in queue for leader lock."); } }