001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.ha;
020
021import java.io.IOException;
022import java.util.Arrays;
023import java.util.List;
024import java.util.concurrent.CountDownLatch;
025import java.util.concurrent.TimeUnit;
026import java.util.concurrent.locks.Lock;
027import java.util.concurrent.locks.ReentrantLock;
028
029import org.apache.commons.logging.Log;
030import org.apache.commons.logging.LogFactory;
031import org.apache.hadoop.HadoopIllegalArgumentException;
032import org.apache.hadoop.classification.InterfaceAudience;
033import org.apache.hadoop.classification.InterfaceStability;
034import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
035import org.apache.hadoop.util.StringUtils;
036import org.apache.zookeeper.data.ACL;
037import org.apache.zookeeper.KeeperException;
038import org.apache.zookeeper.Watcher;
039import org.apache.zookeeper.WatchedEvent;
040import org.apache.zookeeper.Watcher.Event;
041import org.apache.zookeeper.ZKUtil;
042import org.apache.zookeeper.ZooKeeper;
043import org.apache.zookeeper.CreateMode;
044import org.apache.zookeeper.AsyncCallback.*;
045import org.apache.zookeeper.data.Stat;
046import org.apache.zookeeper.KeeperException.Code;
047
048import com.google.common.annotations.VisibleForTesting;
049import com.google.common.base.Preconditions;
050
051/**
052 * 
053 * This class implements a simple library to perform leader election on top of
054 * Apache Zookeeper. Using Zookeeper as a coordination service, leader election
055 * can be performed by atomically creating an ephemeral lock file (znode) on
056 * Zookeeper. The service instance that successfully creates the znode becomes
057 * active and the rest become standbys. <br/>
058 * This election mechanism is only efficient for small number of election
059 * candidates (order of 10's) because contention on single znode by a large
060 * number of candidates can result in Zookeeper overload. <br/>
061 * The elector does not guarantee fencing (protection of shared resources) among
062 * service instances. After it has notified an instance about becoming a leader,
063 * then that instance must ensure that it meets the service consistency
064 * requirements. If it cannot do so, then it is recommended to quit the
065 * election. The application implements the {@link ActiveStandbyElectorCallback}
066 * to interact with the elector
067 */
068@InterfaceAudience.Private
069@InterfaceStability.Evolving
070public class ActiveStandbyElector implements StatCallback, StringCallback {
071
072  /**
073   * Callback interface to interact with the ActiveStandbyElector object. <br/>
074   * The application will be notified with a callback only on state changes
075   * (i.e. there will never be successive calls to becomeActive without an
076   * intermediate call to enterNeutralMode). <br/>
077   * The callbacks will be running on Zookeeper client library threads. The
078   * application should return from these callbacks quickly so as not to impede
079   * Zookeeper client library performance and notifications. The app will
080   * typically remember the state change and return from the callback. It will
081   * then proceed with implementing actions around that state change. It is
082   * possible to be called back again while these actions are in flight and the
083   * app should handle this scenario.
084   */
085  public interface ActiveStandbyElectorCallback {
086    /**
087     * This method is called when the app becomes the active leader.
088     * If the service fails to become active, it should throw
089     * ServiceFailedException. This will cause the elector to
090     * sleep for a short period, then re-join the election.
091     * 
092     * Callback implementations are expected to manage their own
093     * timeouts (e.g. when making an RPC to a remote node).
094     */
095    void becomeActive() throws ServiceFailedException;
096
097    /**
098     * This method is called when the app becomes a standby
099     */
100    void becomeStandby();
101
102    /**
103     * If the elector gets disconnected from Zookeeper and does not know about
104     * the lock state, then it will notify the service via the enterNeutralMode
105     * interface. The service may choose to ignore this or stop doing state
106     * changing operations. Upon reconnection, the elector verifies the leader
107     * status and calls back on the becomeActive and becomeStandby app
108     * interfaces. <br/>
109     * Zookeeper disconnects can happen due to network issues or loss of
110     * Zookeeper quorum. Thus enterNeutralMode can be used to guard against
111     * split-brain issues. In such situations it might be prudent to call
112     * becomeStandby too. However, such state change operations might be
113     * expensive and enterNeutralMode can help guard against doing that for
114     * transient issues.
115     */
116    void enterNeutralMode();
117
118    /**
119     * If there is any fatal error (e.g. wrong ACL's, unexpected Zookeeper
120     * errors or Zookeeper persistent unavailability) then notifyFatalError is
121     * called to notify the app about it.
122     */
123    void notifyFatalError(String errorMessage);
124
125    /**
126     * If an old active has failed, rather than exited gracefully, then
127     * the new active may need to take some fencing actions against it
128     * before proceeding with failover.
129     * 
130     * @param oldActiveData the application data provided by the prior active
131     */
132    void fenceOldActive(byte[] oldActiveData);
133  }
134
135  /**
136   * Name of the lock znode used by the library. Protected for access in test
137   * classes
138   */
139  @VisibleForTesting
140  protected static final String LOCK_FILENAME = "ActiveStandbyElectorLock";
141  @VisibleForTesting
142  protected static final String BREADCRUMB_FILENAME = "ActiveBreadCrumb";
143
144  public static final Log LOG = LogFactory.getLog(ActiveStandbyElector.class);
145
146  static int NUM_RETRIES = 3;
147  private static final int SLEEP_AFTER_FAILURE_TO_BECOME_ACTIVE = 1000;
148
149  private static enum ConnectionState {
150    DISCONNECTED, CONNECTED, TERMINATED
151  };
152
153  static enum State {
154    INIT, ACTIVE, STANDBY, NEUTRAL
155  };
156
157  private State state = State.INIT;
158  private int createRetryCount = 0;
159  private int statRetryCount = 0;
160  private ZooKeeper zkClient;
161  private WatcherWithClientRef watcher;
162  private ConnectionState zkConnectionState = ConnectionState.TERMINATED;
163
164  private final ActiveStandbyElectorCallback appClient;
165  private final String zkHostPort;
166  private final int zkSessionTimeout;
167  private final List<ACL> zkAcl;
168  private final List<ZKAuthInfo> zkAuthInfo;
169  private byte[] appData;
170  private final String zkLockFilePath;
171  private final String zkBreadCrumbPath;
172  private final String znodeWorkingDir;
173
174  private Lock sessionReestablishLockForTests = new ReentrantLock();
175  private boolean wantToBeInElection;
176  
177  /**
178   * Create a new ActiveStandbyElector object <br/>
179   * The elector is created by providing to it the Zookeeper configuration, the
180   * parent znode under which to create the znode and a reference to the
181   * callback interface. <br/>
182   * The parent znode name must be the same for all service instances and
183   * different across services. <br/>
184   * After the leader has been lost, a new leader will be elected after the
185   * session timeout expires. Hence, the app must set this parameter based on
186   * its needs for failure response time. The session timeout must be greater
187   * than the Zookeeper disconnect timeout and is recommended to be 3X that
188   * value to enable Zookeeper to retry transient disconnections. Setting a very
189   * short session timeout may result in frequent transitions between active and
190   * standby states during issues like network outages/GS pauses.
191   * 
192   * @param zookeeperHostPorts
193   *          ZooKeeper hostPort for all ZooKeeper servers
194   * @param zookeeperSessionTimeout
195   *          ZooKeeper session timeout
196   * @param parentZnodeName
197   *          znode under which to create the lock
198   * @param acl
199   *          ZooKeeper ACL's
200   * @param authInfo a list of authentication credentials to add to the
201   *                 ZK connection
202   * @param app
203   *          reference to callback interface object
204   * @throws IOException
205   * @throws HadoopIllegalArgumentException
206   */
207  public ActiveStandbyElector(String zookeeperHostPorts,
208      int zookeeperSessionTimeout, String parentZnodeName, List<ACL> acl,
209      List<ZKAuthInfo> authInfo,
210      ActiveStandbyElectorCallback app) throws IOException,
211      HadoopIllegalArgumentException, KeeperException {
212    if (app == null || acl == null || parentZnodeName == null
213        || zookeeperHostPorts == null || zookeeperSessionTimeout <= 0) {
214      throw new HadoopIllegalArgumentException("Invalid argument");
215    }
216    zkHostPort = zookeeperHostPorts;
217    zkSessionTimeout = zookeeperSessionTimeout;
218    zkAcl = acl;
219    zkAuthInfo = authInfo;
220    appClient = app;
221    znodeWorkingDir = parentZnodeName;
222    zkLockFilePath = znodeWorkingDir + "/" + LOCK_FILENAME;
223    zkBreadCrumbPath = znodeWorkingDir + "/" + BREADCRUMB_FILENAME;    
224
225    // createConnection for future API calls
226    createConnection();
227  }
228
229  /**
230   * To participate in election, the app will call joinElection. The result will
231   * be notified by a callback on either the becomeActive or becomeStandby app
232   * interfaces. <br/>
233   * After this the elector will automatically monitor the leader status and
234   * perform re-election if necessary<br/>
235   * The app could potentially start off in standby mode and ignore the
236   * becomeStandby call.
237   * 
238   * @param data
239   *          to be set by the app. non-null data must be set.
240   * @throws HadoopIllegalArgumentException
241   *           if valid data is not supplied
242   */
243  public synchronized void joinElection(byte[] data)
244      throws HadoopIllegalArgumentException {
245    
246    if (data == null) {
247      throw new HadoopIllegalArgumentException("data cannot be null");
248    }
249    
250    if (wantToBeInElection) {
251      LOG.info("Already in election. Not re-connecting.");
252      return;
253    }
254
255    appData = new byte[data.length];
256    System.arraycopy(data, 0, appData, 0, data.length);
257
258    LOG.debug("Attempting active election for " + this);
259    joinElectionInternal();
260  }
261  
262  /**
263   * @return true if the configured parent znode exists
264   */
265  public synchronized boolean parentZNodeExists()
266      throws IOException, InterruptedException {
267    Preconditions.checkState(zkClient != null);
268    try {
269      return zkClient.exists(znodeWorkingDir, false) != null;
270    } catch (KeeperException e) {
271      throw new IOException("Couldn't determine existence of znode '" +
272          znodeWorkingDir + "'", e);
273    }
274  }
275
276  /**
277   * Utility function to ensure that the configured base znode exists.
278   * This recursively creates the znode as well as all of its parents.
279   */
280  public synchronized void ensureParentZNode()
281      throws IOException, InterruptedException {
282    Preconditions.checkState(!wantToBeInElection,
283        "ensureParentZNode() may not be called while in the election");
284
285    String pathParts[] = znodeWorkingDir.split("/");
286    Preconditions.checkArgument(pathParts.length >= 1 &&
287        pathParts[0].isEmpty(),
288        "Invalid path: %s", znodeWorkingDir);
289    
290    StringBuilder sb = new StringBuilder();
291    for (int i = 1; i < pathParts.length; i++) {
292      sb.append("/").append(pathParts[i]);
293      String prefixPath = sb.toString();
294      LOG.debug("Ensuring existence of " + prefixPath);
295      try {
296        createWithRetries(prefixPath, new byte[]{}, zkAcl, CreateMode.PERSISTENT);
297      } catch (KeeperException e) {
298        if (isNodeExists(e.code())) {
299          // This is OK - just ensuring existence.
300          continue;
301        } else {
302          throw new IOException("Couldn't create " + prefixPath, e);
303        }
304      }
305    }
306    
307    LOG.info("Successfully created " + znodeWorkingDir + " in ZK.");
308  }
309  
310  /**
311   * Clear all of the state held within the parent ZNode.
312   * This recursively deletes everything within the znode as well as the
313   * parent znode itself. It should only be used when it's certain that
314   * no electors are currently participating in the election.
315   */
316  public synchronized void clearParentZNode()
317      throws IOException, InterruptedException {
318    Preconditions.checkState(!wantToBeInElection,
319        "clearParentZNode() may not be called while in the election");
320
321    try {
322      LOG.info("Recursively deleting " + znodeWorkingDir + " from ZK...");
323
324      zkDoWithRetries(new ZKAction<Void>() {
325        @Override
326        public Void run() throws KeeperException, InterruptedException {
327          ZKUtil.deleteRecursive(zkClient, znodeWorkingDir);
328          return null;
329        }
330      });
331    } catch (KeeperException e) {
332      throw new IOException("Couldn't clear parent znode " + znodeWorkingDir,
333          e);
334    }
335    LOG.info("Successfully deleted " + znodeWorkingDir + " from ZK.");
336  }
337
338
339  /**
340   * Any service instance can drop out of the election by calling quitElection. 
341   * <br/>
342   * This will lose any leader status, if held, and stop monitoring of the lock
343   * node. <br/>
344   * If the instance wants to participate in election again, then it needs to
345   * call joinElection(). <br/>
346   * This allows service instances to take themselves out of rotation for known
347   * impending unavailable states (e.g. long GC pause or software upgrade).
348   * 
349   * @param needFence true if the underlying daemon may need to be fenced
350   * if a failover occurs due to dropping out of the election.
351   */
352  public synchronized void quitElection(boolean needFence) {
353    LOG.info("Yielding from election");
354    if (!needFence && state == State.ACTIVE) {
355      // If active is gracefully going back to standby mode, remove
356      // our permanent znode so no one fences us.
357      tryDeleteOwnBreadCrumbNode();
358    }
359    reset();
360    wantToBeInElection = false;
361  }
362
363  /**
364   * Exception thrown when there is no active leader
365   */
366  public static class ActiveNotFoundException extends Exception {
367    private static final long serialVersionUID = 3505396722342846462L;
368  }
369
370  /**
371   * get data set by the active leader
372   * 
373   * @return data set by the active instance
374   * @throws ActiveNotFoundException
375   *           when there is no active leader
376   * @throws KeeperException
377   *           other zookeeper operation errors
378   * @throws InterruptedException
379   * @throws IOException
380   *           when ZooKeeper connection could not be established
381   */
382  public synchronized byte[] getActiveData() throws ActiveNotFoundException,
383      KeeperException, InterruptedException, IOException {
384    try {
385      if (zkClient == null) {
386        createConnection();
387      }
388      Stat stat = new Stat();
389      return getDataWithRetries(zkLockFilePath, false, stat);
390    } catch(KeeperException e) {
391      Code code = e.code();
392      if (isNodeDoesNotExist(code)) {
393        // handle the commonly expected cases that make sense for us
394        throw new ActiveNotFoundException();
395      } else {
396        throw e;
397      }
398    }
399  }
400
401  /**
402   * interface implementation of Zookeeper callback for create
403   */
404  @Override
405  public synchronized void processResult(int rc, String path, Object ctx,
406      String name) {
407    if (isStaleClient(ctx)) return;
408    LOG.debug("CreateNode result: " + rc + " for path: " + path
409        + " connectionState: " + zkConnectionState +
410        "  for " + this);
411
412    Code code = Code.get(rc);
413    if (isSuccess(code)) {
414      // we successfully created the znode. we are the leader. start monitoring
415      if (becomeActive()) {
416        monitorActiveStatus();
417      } else {
418        reJoinElectionAfterFailureToBecomeActive();
419      }
420      return;
421    }
422
423    if (isNodeExists(code)) {
424      if (createRetryCount == 0) {
425        // znode exists and we did not retry the operation. so a different
426        // instance has created it. become standby and monitor lock.
427        becomeStandby();
428      }
429      // if we had retried then the znode could have been created by our first
430      // attempt to the server (that we lost) and this node exists response is
431      // for the second attempt. verify this case via ephemeral node owner. this
432      // will happen on the callback for monitoring the lock.
433      monitorActiveStatus();
434      return;
435    }
436
437    String errorMessage = "Received create error from Zookeeper. code:"
438        + code.toString() + " for path " + path;
439    LOG.debug(errorMessage);
440
441    if (shouldRetry(code)) {
442      if (createRetryCount < NUM_RETRIES) {
443        LOG.debug("Retrying createNode createRetryCount: " + createRetryCount);
444        ++createRetryCount;
445        createLockNodeAsync();
446        return;
447      }
448      errorMessage = errorMessage
449          + ". Not retrying further znode create connection errors.";
450    } else if (isSessionExpired(code)) {
451      // This isn't fatal - the client Watcher will re-join the election
452      LOG.warn("Lock acquisition failed because session was lost");
453      return;
454    }
455
456    fatalError(errorMessage);
457  }
458
459  /**
460   * interface implementation of Zookeeper callback for monitor (exists)
461   */
462  @Override
463  public synchronized void processResult(int rc, String path, Object ctx,
464      Stat stat) {
465    if (isStaleClient(ctx)) return;
466    
467    assert wantToBeInElection :
468        "Got a StatNode result after quitting election";
469    
470    LOG.debug("StatNode result: " + rc + " for path: " + path
471        + " connectionState: " + zkConnectionState + " for " + this);
472        
473
474    Code code = Code.get(rc);
475    if (isSuccess(code)) {
476      // the following owner check completes verification in case the lock znode
477      // creation was retried
478      if (stat.getEphemeralOwner() == zkClient.getSessionId()) {
479        // we own the lock znode. so we are the leader
480        if (!becomeActive()) {
481          reJoinElectionAfterFailureToBecomeActive();
482        }
483      } else {
484        // we dont own the lock znode. so we are a standby.
485        becomeStandby();
486      }
487      // the watch set by us will notify about changes
488      return;
489    }
490
491    if (isNodeDoesNotExist(code)) {
492      // the lock znode disappeared before we started monitoring it
493      enterNeutralMode();
494      joinElectionInternal();
495      return;
496    }
497
498    String errorMessage = "Received stat error from Zookeeper. code:"
499        + code.toString();
500    LOG.debug(errorMessage);
501
502    if (shouldRetry(code)) {
503      if (statRetryCount < NUM_RETRIES) {
504        ++statRetryCount;
505        monitorLockNodeAsync();
506        return;
507      }
508      errorMessage = errorMessage
509          + ". Not retrying further znode monitoring connection errors.";
510    } else if (isSessionExpired(code)) {
511      // This isn't fatal - the client Watcher will re-join the election
512      LOG.warn("Lock monitoring failed because session was lost");
513      return;
514    }
515
516    fatalError(errorMessage);
517  }
518
519  /**
520   * We failed to become active. Re-join the election, but
521   * sleep for a few seconds after terminating our existing
522   * session, so that other nodes have a chance to become active.
523   * The failure to become active is already logged inside
524   * becomeActive().
525   */
526  private void reJoinElectionAfterFailureToBecomeActive() {
527    reJoinElection(SLEEP_AFTER_FAILURE_TO_BECOME_ACTIVE);
528  }
529
530  /**
531   * interface implementation of Zookeeper watch events (connection and node),
532   * proxied by {@link WatcherWithClientRef}.
533   */
534  synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) {
535    Event.EventType eventType = event.getType();
536    if (isStaleClient(zk)) return;
537    LOG.debug("Watcher event type: " + eventType + " with state:"
538        + event.getState() + " for path:" + event.getPath()
539        + " connectionState: " + zkConnectionState
540        + " for " + this);
541
542    if (eventType == Event.EventType.None) {
543      // the connection state has changed
544      switch (event.getState()) {
545      case SyncConnected:
546        LOG.info("Session connected.");
547        // if the listener was asked to move to safe state then it needs to
548        // be undone
549        ConnectionState prevConnectionState = zkConnectionState;
550        zkConnectionState = ConnectionState.CONNECTED;
551        if (prevConnectionState == ConnectionState.DISCONNECTED &&
552            wantToBeInElection) {
553          monitorActiveStatus();
554        }
555        break;
556      case Disconnected:
557        LOG.info("Session disconnected. Entering neutral mode...");
558
559        // ask the app to move to safe state because zookeeper connection
560        // is not active and we dont know our state
561        zkConnectionState = ConnectionState.DISCONNECTED;
562        enterNeutralMode();
563        break;
564      case Expired:
565        // the connection got terminated because of session timeout
566        // call listener to reconnect
567        LOG.info("Session expired. Entering neutral mode and rejoining...");
568        enterNeutralMode();
569        reJoinElection(0);
570        break;
571      case SaslAuthenticated:
572        LOG.info("Successfully authenticated to ZooKeeper using SASL.");
573        break;
574      default:
575        fatalError("Unexpected Zookeeper watch event state: "
576            + event.getState());
577        break;
578      }
579
580      return;
581    }
582
583    // a watch on lock path in zookeeper has fired. so something has changed on
584    // the lock. ideally we should check that the path is the same as the lock
585    // path but trusting zookeeper for now
586    String path = event.getPath();
587    if (path != null) {
588      switch (eventType) {
589      case NodeDeleted:
590        if (state == State.ACTIVE) {
591          enterNeutralMode();
592        }
593        joinElectionInternal();
594        break;
595      case NodeDataChanged:
596        monitorActiveStatus();
597        break;
598      default:
599        LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
600        monitorActiveStatus();
601      }
602
603      return;
604    }
605
606    // some unexpected error has occurred
607    fatalError("Unexpected watch error from Zookeeper");
608  }
609
610  /**
611   * Get a new zookeeper client instance. protected so that test class can
612   * inherit and pass in a mock object for zookeeper
613   * 
614   * @return new zookeeper client instance
615   * @throws IOException
616   * @throws KeeperException zookeeper connectionloss exception
617   */
618  protected synchronized ZooKeeper getNewZooKeeper() throws IOException,
619      KeeperException {
620    
621    // Unfortunately, the ZooKeeper constructor connects to ZooKeeper and
622    // may trigger the Connected event immediately. So, if we register the
623    // watcher after constructing ZooKeeper, we may miss that event. Instead,
624    // we construct the watcher first, and have it block any events it receives
625    // before we can set its ZooKeeper reference.
626    watcher = new WatcherWithClientRef();
627    ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher);
628    watcher.setZooKeeperRef(zk);
629
630    // Wait for the asynchronous success/failure. This may throw an exception
631    // if we don't connect within the session timeout.
632    watcher.waitForZKConnectionEvent(zkSessionTimeout);
633    
634    for (ZKAuthInfo auth : zkAuthInfo) {
635      zk.addAuthInfo(auth.getScheme(), auth.getAuth());
636    }
637    return zk;
638  }
639
640  private void fatalError(String errorMessage) {
641    LOG.fatal(errorMessage);
642    reset();
643    appClient.notifyFatalError(errorMessage);
644  }
645
646  private void monitorActiveStatus() {
647    assert wantToBeInElection;
648    LOG.debug("Monitoring active leader for " + this);
649    statRetryCount = 0;
650    monitorLockNodeAsync();
651  }
652
653  private void joinElectionInternal() {
654    Preconditions.checkState(appData != null,
655        "trying to join election without any app data");
656    if (zkClient == null) {
657      if (!reEstablishSession()) {
658        fatalError("Failed to reEstablish connection with ZooKeeper");
659        return;
660      }
661    }
662
663    createRetryCount = 0;
664    wantToBeInElection = true;
665    createLockNodeAsync();
666  }
667
668  private void reJoinElection(int sleepTime) {
669    LOG.info("Trying to re-establish ZK session");
670    
671    // Some of the test cases rely on expiring the ZK sessions and
672    // ensuring that the other node takes over. But, there's a race
673    // where the original lease holder could reconnect faster than the other
674    // thread manages to take the lock itself. This lock allows the
675    // tests to block the reconnection. It's a shame that this leaked
676    // into non-test code, but the lock is only acquired here so will never
677    // be contended.
678    sessionReestablishLockForTests.lock();
679    try {
680      terminateConnection();
681      sleepFor(sleepTime);
682      // Should not join election even before the SERVICE is reported
683      // as HEALTHY from ZKFC monitoring.
684      if (appData != null) {
685        joinElectionInternal();
686      } else {
687        LOG.info("Not joining election since service has not yet been " +
688            "reported as healthy.");
689      }
690    } finally {
691      sessionReestablishLockForTests.unlock();
692    }
693  }
694
695  /**
696   * Sleep for the given number of milliseconds.
697   * This is non-static, and separated out, so that unit tests
698   * can override the behavior not to sleep.
699   */
700  @VisibleForTesting
701  protected void sleepFor(int sleepMs) {
702    if (sleepMs > 0) {
703      try {
704        Thread.sleep(sleepMs);
705      } catch (InterruptedException e) {
706        Thread.currentThread().interrupt();
707      }
708    }
709  }
710
711  @VisibleForTesting
712  void preventSessionReestablishmentForTests() {
713    sessionReestablishLockForTests.lock();
714  }
715  
716  @VisibleForTesting
717  void allowSessionReestablishmentForTests() {
718    sessionReestablishLockForTests.unlock();
719  }
720  
721  @VisibleForTesting
722  synchronized long getZKSessionIdForTests() {
723    if (zkClient != null) {
724      return zkClient.getSessionId();
725    } else {
726      return -1;
727    }
728  }
729  
730  @VisibleForTesting
731  synchronized State getStateForTests() {
732    return state;
733  }
734
735  private boolean reEstablishSession() {
736    int connectionRetryCount = 0;
737    boolean success = false;
738    while(!success && connectionRetryCount < NUM_RETRIES) {
739      LOG.debug("Establishing zookeeper connection for " + this);
740      try {
741        createConnection();
742        success = true;
743      } catch(IOException e) {
744        LOG.warn(e);
745        sleepFor(5000);
746      } catch(KeeperException e) {
747        LOG.warn(e);
748        sleepFor(5000);
749      }
750      ++connectionRetryCount;
751    }
752    return success;
753  }
754
755  private void createConnection() throws IOException, KeeperException {
756    if (zkClient != null) {
757      try {
758        zkClient.close();
759      } catch (InterruptedException e) {
760        throw new IOException("Interrupted while closing ZK",
761            e);
762      }
763      zkClient = null;
764      watcher = null;
765    }
766    zkClient = getNewZooKeeper();
767    LOG.debug("Created new connection for " + this);
768  }
769
770  @InterfaceAudience.Private
771  public synchronized void terminateConnection() {
772    if (zkClient == null) {
773      return;
774    }
775    LOG.debug("Terminating ZK connection for " + this);
776    ZooKeeper tempZk = zkClient;
777    zkClient = null;
778    watcher = null;
779    try {
780      tempZk.close();
781    } catch(InterruptedException e) {
782      LOG.warn(e);
783    }
784    zkConnectionState = ConnectionState.TERMINATED;
785    wantToBeInElection = false;
786  }
787
788  private void reset() {
789    state = State.INIT;
790    terminateConnection();
791  }
792
793  private boolean becomeActive() {
794    assert wantToBeInElection;
795    if (state == State.ACTIVE) {
796      // already active
797      return true;
798    }
799    try {
800      Stat oldBreadcrumbStat = fenceOldActive();
801      writeBreadCrumbNode(oldBreadcrumbStat);
802      
803      LOG.debug("Becoming active for " + this);
804      appClient.becomeActive();
805      state = State.ACTIVE;
806      return true;
807    } catch (Exception e) {
808      LOG.warn("Exception handling the winning of election", e);
809      // Caller will handle quitting and rejoining the election.
810      return false;
811    }
812  }
813
814  /**
815   * Write the "ActiveBreadCrumb" node, indicating that this node may need
816   * to be fenced on failover.
817   * @param oldBreadcrumbStat 
818   */
819  private void writeBreadCrumbNode(Stat oldBreadcrumbStat)
820      throws KeeperException, InterruptedException {
821    Preconditions.checkState(appData != null, "no appdata");
822    
823    LOG.info("Writing znode " + zkBreadCrumbPath +
824        " to indicate that the local node is the most recent active...");
825    if (oldBreadcrumbStat == null) {
826      // No previous active, just create the node
827      createWithRetries(zkBreadCrumbPath, appData, zkAcl,
828        CreateMode.PERSISTENT);
829    } else {
830      // There was a previous active, update the node
831      setDataWithRetries(zkBreadCrumbPath, appData, oldBreadcrumbStat.getVersion());
832    }
833  }
834  
835  /**
836   * Try to delete the "ActiveBreadCrumb" node when gracefully giving up
837   * active status.
838   * If this fails, it will simply warn, since the graceful release behavior
839   * is only an optimization.
840   */
841  private void tryDeleteOwnBreadCrumbNode() {
842    assert state == State.ACTIVE;
843    LOG.info("Deleting bread-crumb of active node...");
844    
845    // Sanity check the data. This shouldn't be strictly necessary,
846    // but better to play it safe.
847    Stat stat = new Stat();
848    byte[] data = null;
849    try {
850      data = zkClient.getData(zkBreadCrumbPath, false, stat);
851
852      if (!Arrays.equals(data, appData)) {
853        throw new IllegalStateException(
854            "We thought we were active, but in fact " +
855            "the active znode had the wrong data: " +
856            StringUtils.byteToHexString(data) + " (stat=" + stat + ")");
857      }
858      
859      deleteWithRetries(zkBreadCrumbPath, stat.getVersion());
860    } catch (Exception e) {
861      LOG.warn("Unable to delete our own bread-crumb of being active at " +
862          zkBreadCrumbPath + ": " + e.getLocalizedMessage() + ". " +
863          "Expecting to be fenced by the next active.");
864    }
865  }
866
867  /**
868   * If there is a breadcrumb node indicating that another node may need
869   * fencing, try to fence that node.
870   * @return the Stat of the breadcrumb node that was read, or null
871   * if no breadcrumb node existed
872   */
873  private Stat fenceOldActive() throws InterruptedException, KeeperException {
874    final Stat stat = new Stat();
875    byte[] data;
876    LOG.info("Checking for any old active which needs to be fenced...");
877    try {
878      data = zkDoWithRetries(new ZKAction<byte[]>() {
879        @Override
880        public byte[] run() throws KeeperException, InterruptedException {
881          return zkClient.getData(zkBreadCrumbPath, false, stat);
882        }
883      });
884    } catch (KeeperException ke) {
885      if (isNodeDoesNotExist(ke.code())) {
886        LOG.info("No old node to fence");
887        return null;
888      }
889      
890      // If we failed to read for any other reason, then likely we lost
891      // our session, or we don't have permissions, etc. In any case,
892      // we probably shouldn't become active, and failing the whole
893      // thing is the best bet.
894      throw ke;
895    }
896
897    LOG.info("Old node exists: " + StringUtils.byteToHexString(data));
898    if (Arrays.equals(data, appData)) {
899      LOG.info("But old node has our own data, so don't need to fence it.");
900    } else {
901      appClient.fenceOldActive(data);
902    }
903    return stat;
904  }
905
906  private void becomeStandby() {
907    if (state != State.STANDBY) {
908      LOG.debug("Becoming standby for " + this);
909      state = State.STANDBY;
910      appClient.becomeStandby();
911    }
912  }
913
914  private void enterNeutralMode() {
915    if (state != State.NEUTRAL) {
916      LOG.debug("Entering neutral mode for " + this);
917      state = State.NEUTRAL;
918      appClient.enterNeutralMode();
919    }
920  }
921
922  private void createLockNodeAsync() {
923    zkClient.create(zkLockFilePath, appData, zkAcl, CreateMode.EPHEMERAL,
924        this, zkClient);
925  }
926
927  private void monitorLockNodeAsync() {
928    zkClient.exists(zkLockFilePath, 
929        watcher, this,
930        zkClient);
931  }
932
933  private String createWithRetries(final String path, final byte[] data,
934      final List<ACL> acl, final CreateMode mode)
935      throws InterruptedException, KeeperException {
936    return zkDoWithRetries(new ZKAction<String>() {
937      @Override
938      public String run() throws KeeperException, InterruptedException {
939        return zkClient.create(path, data, acl, mode);
940      }
941    });
942  }
943
944  private byte[] getDataWithRetries(final String path, final boolean watch,
945      final Stat stat) throws InterruptedException, KeeperException {
946    return zkDoWithRetries(new ZKAction<byte[]>() {
947      @Override
948      public byte[] run() throws KeeperException, InterruptedException {
949        return zkClient.getData(path, watch, stat);
950      }
951    });
952  }
953
954  private Stat setDataWithRetries(final String path, final byte[] data,
955      final int version) throws InterruptedException, KeeperException {
956    return zkDoWithRetries(new ZKAction<Stat>() {
957      @Override
958      public Stat run() throws KeeperException, InterruptedException {
959        return zkClient.setData(path, data, version);
960      }
961    });
962  }
963  
964  private void deleteWithRetries(final String path, final int version)
965      throws KeeperException, InterruptedException {
966    zkDoWithRetries(new ZKAction<Void>() {
967      @Override
968      public Void run() throws KeeperException, InterruptedException {
969        zkClient.delete(path, version);
970        return null;
971      }
972    });
973  }
974
975  private static <T> T zkDoWithRetries(ZKAction<T> action)
976      throws KeeperException, InterruptedException {
977    int retry = 0;
978    while (true) {
979      try {
980        return action.run();
981      } catch (KeeperException ke) {
982        if (shouldRetry(ke.code()) && ++retry < NUM_RETRIES) {
983          continue;
984        }
985        throw ke;
986      }
987    }
988  }
989
990  private interface ZKAction<T> {
991    T run() throws KeeperException, InterruptedException; 
992  }
993  
994  /**
995   * The callbacks and watchers pass a reference to the ZK client
996   * which made the original call. We don't want to take action
997   * based on any callbacks from prior clients after we quit
998   * the election.
999   * @param ctx the ZK client passed into the watcher
1000   * @return true if it matches the current client
1001   */
1002  private synchronized boolean isStaleClient(Object ctx) {
1003    Preconditions.checkNotNull(ctx);
1004    if (zkClient != (ZooKeeper)ctx) {
1005      LOG.warn("Ignoring stale result from old client with sessionId " +
1006          String.format("0x%08x", ((ZooKeeper)ctx).getSessionId()));
1007      return true;
1008    }
1009    return false;
1010  }
1011
1012  /**
1013   * Watcher implementation which keeps a reference around to the
1014   * original ZK connection, and passes it back along with any
1015   * events.
1016   */
1017  private final class WatcherWithClientRef implements Watcher {
1018    private ZooKeeper zk;
1019    
1020    /**
1021     * Latch fired whenever any event arrives. This is used in order
1022     * to wait for the Connected event when the client is first created.
1023     */
1024    private CountDownLatch hasReceivedEvent = new CountDownLatch(1);
1025
1026    /**
1027     * Latch used to wait until the reference to ZooKeeper is set.
1028     */
1029    private CountDownLatch hasSetZooKeeper = new CountDownLatch(1);
1030
1031    /**
1032     * Waits for the next event from ZooKeeper to arrive.
1033     * 
1034     * @param connectionTimeoutMs zookeeper connection timeout in milliseconds
1035     * @throws KeeperException if the connection attempt times out. This will
1036     * be a ZooKeeper ConnectionLoss exception code.
1037     * @throws IOException if interrupted while connecting to ZooKeeper
1038     */
1039    private void waitForZKConnectionEvent(int connectionTimeoutMs)
1040        throws KeeperException, IOException {
1041      try {
1042        if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) {
1043          LOG.error("Connection timed out: couldn't connect to ZooKeeper in "
1044              + connectionTimeoutMs + " milliseconds");
1045          zk.close();
1046          throw KeeperException.create(Code.CONNECTIONLOSS);
1047        }
1048      } catch (InterruptedException e) {
1049        Thread.currentThread().interrupt();
1050        throw new IOException(
1051            "Interrupted when connecting to zookeeper server", e);
1052      }
1053    }
1054
1055    private void setZooKeeperRef(ZooKeeper zk) {
1056      Preconditions.checkState(this.zk == null,
1057          "zk already set -- must be set exactly once");
1058      this.zk = zk;
1059      hasSetZooKeeper.countDown();
1060    }
1061
1062    @Override
1063    public void process(WatchedEvent event) {
1064      hasReceivedEvent.countDown();
1065      try {
1066        hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS);
1067        ActiveStandbyElector.this.processWatchEvent(
1068            zk, event);
1069      } catch (Throwable t) {
1070        fatalError(
1071            "Failed to process watcher event " + event + ": " +
1072            StringUtils.stringifyException(t));
1073      }
1074    }
1075  }
1076
1077  private static boolean isSuccess(Code code) {
1078    return (code == Code.OK);
1079  }
1080
1081  private static boolean isNodeExists(Code code) {
1082    return (code == Code.NODEEXISTS);
1083  }
1084
1085  private static boolean isNodeDoesNotExist(Code code) {
1086    return (code == Code.NONODE);
1087  }
1088  
1089  private static boolean isSessionExpired(Code code) {
1090    return (code == Code.SESSIONEXPIRED);
1091  }
1092
1093  private static boolean shouldRetry(Code code) {
1094    switch (code) {
1095    case CONNECTIONLOSS:
1096    case OPERATIONTIMEOUT:
1097      return true;
1098    }
1099    return false;
1100  }
1101  
1102  @Override
1103  public String toString() {
1104    return "elector id=" + System.identityHashCode(this) +
1105      " appData=" +
1106      ((appData == null) ? "null" : StringUtils.byteToHexString(appData)) + 
1107      " cb=" + appClient;
1108  }
1109}