View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.zookeeper;
20  
21  import java.util.concurrent.atomic.AtomicBoolean;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.Stoppable;
27  import org.apache.hadoop.hbase.util.Bytes;
28  import org.apache.zookeeper.KeeperException;
29  
30  /**
31   * Handles coordination of a single "leader" instance among many possible
32   * candidates.  The first {@link ZKLeaderManager} to successfully create
33   * the given znode becomes the leader, allowing the instance to continue
34   * with whatever processing must be protected.  Other {@link ZKLeaderManager}
35   * instances will wait to be notified of changes to the leader znode.
36   * If the current master instance fails, the ephemeral leader znode will
37   * be removed, and all waiting instances will be notified, with the race
38   * to claim the leader znode beginning all over again.
39   * @deprecated Not used
40   */
41  @Deprecated
42  @InterfaceAudience.Private
43  public class ZKLeaderManager extends ZooKeeperListener {
44    private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
45  
46    private final Object lock = new Object();
47    private final AtomicBoolean leaderExists = new AtomicBoolean();
48    private String leaderZNode;
49    private byte[] nodeId;
50    private Stoppable candidate;
51  
52    public ZKLeaderManager(ZooKeeperWatcher watcher, String leaderZNode,
53        byte[] identifier, Stoppable candidate) {
54      super(watcher);
55      this.leaderZNode = leaderZNode;
56      this.nodeId = identifier;
57      this.candidate = candidate;
58    }
59  
60    public void start() {
61      try {
62        watcher.registerListener(this);
63        String parent = ZKUtil.getParent(leaderZNode);
64        if (ZKUtil.checkExists(watcher, parent) < 0) {
65          ZKUtil.createWithParents(watcher, parent);
66        }
67      } catch (KeeperException ke) {
68        watcher.abort("Unhandled zk exception when starting", ke);
69        candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
70      }
71    }
72  
73    @Override
74    public void nodeCreated(String path) {
75      if (leaderZNode.equals(path) && !candidate.isStopped()) {
76        handleLeaderChange();
77      }
78    }
79  
80    @Override
81    public void nodeDeleted(String path) {
82      if (leaderZNode.equals(path) && !candidate.isStopped()) {
83        handleLeaderChange();
84      }
85    }
86  
87    private void handleLeaderChange() {
88      try {
89        synchronized(lock) {
90          if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
91            LOG.info("Found new leader for znode: "+leaderZNode);
92            leaderExists.set(true);
93          } else {
94            LOG.info("Leader change, but no new leader found");
95            leaderExists.set(false);
96            lock.notifyAll();
97          }
98        }
99      } catch (KeeperException ke) {
100       watcher.abort("ZooKeeper error checking for leader znode", ke);
101       candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
102     }
103   }
104 
105   /**
106    * Blocks until this instance has claimed the leader ZNode in ZooKeeper
107    */
108   public void waitToBecomeLeader() {
109     while (!candidate.isStopped()) {
110       try {
111         if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
112           // claimed the leader znode
113           leaderExists.set(true);
114           if (LOG.isDebugEnabled()) {
115             LOG.debug("Claimed the leader znode as '"+
116                 Bytes.toStringBinary(nodeId)+"'");
117           }
118           return;
119         }
120 
121         // if claiming the node failed, there should be another existing node
122         byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
123         if (currentId != null && Bytes.equals(currentId, nodeId)) {
124           // claimed with our ID, but we didn't grab it, possibly restarted?
125           LOG.info("Found existing leader with our ID ("+
126               Bytes.toStringBinary(nodeId)+"), removing");
127           ZKUtil.deleteNode(watcher, leaderZNode);
128           leaderExists.set(false);
129         } else {
130           LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
131           leaderExists.set(true);
132         }
133       } catch (KeeperException ke) {
134         watcher.abort("Unexpected error from ZK, stopping candidate", ke);
135         candidate.stop("Unexpected error from ZK: "+ke.getMessage());
136         return;
137       }
138 
139       // wait for next chance
140       synchronized(lock) {
141         while (leaderExists.get() && !candidate.isStopped()) {
142           try {
143             lock.wait();
144           } catch (InterruptedException ie) {
145             LOG.debug("Interrupted waiting on leader", ie);
146           }
147         }
148       }
149     }
150   }
151 
152   /**
153    * Removes the leader znode, if it is currently claimed by this instance.
154    */
155   public void stepDownAsLeader() {
156     try {
157       synchronized(lock) {
158         if (!leaderExists.get()) {
159           return;
160         }
161         byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
162         if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
163           LOG.info("Stepping down as leader");
164           ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
165           leaderExists.set(false);
166         } else {
167           LOG.info("Not current leader, no need to step down");
168         }
169       }
170     } catch (KeeperException ke) {
171       watcher.abort("Unhandled zookeeper exception removing leader node", ke);
172       candidate.stop("Unhandled zookeeper exception removing leader node: "
173           + ke.getMessage());
174     } catch (InterruptedException e) {
175       watcher.abort("Unhandled zookeeper exception removing leader node", e);
176       candidate.stop("Unhandled zookeeper exception removing leader node: "
177           + e.getMessage());
178     }
179   }
180 
181   public boolean hasLeader() {
182     return leaderExists.get();
183   }
184 }