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 org.apache.commons.logging.Log;
22  import org.apache.commons.logging.LogFactory;
23  import org.apache.hadoop.classification.InterfaceAudience;
24  import org.apache.hadoop.classification.InterfaceStability;
25  import org.apache.hadoop.hbase.Stoppable;
26  import org.apache.hadoop.hbase.util.Bytes;
27  import org.apache.zookeeper.KeeperException;
28  
29  import java.util.concurrent.atomic.AtomicBoolean;
30  
31  /**
32   * Handles coordination of a single "leader" instance among many possible
33   * candidates.  The first {@link ZKLeaderManager} to successfully create
34   * the given znode becomes the leader, allowing the instance to continue
35   * with whatever processing must be protected.  Other {@link ZKLeaderManager}
36   * instances will wait to be notified of changes to the leader znode.
37   * If the current master instance fails, the ephemeral leader znode will
38   * be removed, and all waiting instances will be notified, with the race
39   * to claim the leader znode beginning all over again.
40   * @deprecated Not used
41   */
42  @Deprecated
43  @InterfaceAudience.Public
44  @InterfaceStability.Evolving
45  public class ZKLeaderManager extends ZooKeeperListener {
46    private static Log LOG = LogFactory.getLog(ZKLeaderManager.class);
47  
48    private final AtomicBoolean leaderExists = new AtomicBoolean();
49    private String leaderZNode;
50    private byte[] nodeId;
51    private Stoppable candidate;
52  
53    public ZKLeaderManager(ZooKeeperWatcher watcher, String leaderZNode,
54        byte[] identifier, Stoppable candidate) {
55      super(watcher);
56      this.leaderZNode = leaderZNode;
57      this.nodeId = identifier;
58      this.candidate = candidate;
59    }
60  
61    public void start() {
62      try {
63        watcher.registerListener(this);
64        String parent = ZKUtil.getParent(leaderZNode);
65        if (ZKUtil.checkExists(watcher, parent) < 0) {
66          ZKUtil.createWithParents(watcher, parent);
67        }
68      } catch (KeeperException ke) {
69        watcher.abort("Unhandled zk exception when starting", ke);
70        candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
71      }
72    }
73  
74    @Override
75    public void nodeCreated(String path) {
76      if (leaderZNode.equals(path) && !candidate.isStopped()) {
77        handleLeaderChange();
78      }
79    }
80  
81    @Override
82    public void nodeDeleted(String path) {
83      if (leaderZNode.equals(path) && !candidate.isStopped()) {
84        handleLeaderChange();
85      }
86    }
87  
88    private void handleLeaderChange() {
89      try {
90        synchronized(leaderExists) {
91          if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
92            LOG.info("Found new leader for znode: "+leaderZNode);
93            leaderExists.set(true);
94          } else {
95            LOG.info("Leader change, but no new leader found");
96            leaderExists.set(false);
97            leaderExists.notifyAll();
98          }
99        }
100     } catch (KeeperException ke) {
101       watcher.abort("ZooKeeper error checking for leader znode", ke);
102       candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
103     }
104   }
105 
106   /**
107    * Blocks until this instance has claimed the leader ZNode in ZooKeeper
108    */
109   public void waitToBecomeLeader() {
110     while (!candidate.isStopped()) {
111       try {
112         if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
113           // claimed the leader znode
114           leaderExists.set(true);
115           if (LOG.isDebugEnabled()) {
116             LOG.debug("Claimed the leader znode as '"+
117                 Bytes.toStringBinary(nodeId)+"'");
118           }
119           return;
120         }
121 
122         // if claiming the node failed, there should be another existing node
123         byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
124         if (currentId != null && Bytes.equals(currentId, nodeId)) {
125           // claimed with our ID, but we didn't grab it, possibly restarted?
126           LOG.info("Found existing leader with our ID ("+
127               Bytes.toStringBinary(nodeId)+"), removing");
128           ZKUtil.deleteNode(watcher, leaderZNode);
129           leaderExists.set(false);
130         } else {
131           LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
132           leaderExists.set(true);
133         }
134       } catch (KeeperException ke) {
135         watcher.abort("Unexpected error from ZK, stopping candidate", ke);
136         candidate.stop("Unexpected error from ZK: "+ke.getMessage());
137         return;
138       }
139 
140       // wait for next chance
141       synchronized(leaderExists) {
142         while (leaderExists.get() && !candidate.isStopped()) {
143           try {
144             leaderExists.wait();
145           } catch (InterruptedException ie) {
146             LOG.debug("Interrupted waiting on leader", ie);
147           }
148         }
149       }
150     }
151   }
152 
153   /**
154    * Removes the leader znode, if it is currently claimed by this instance.
155    */
156   public void stepDownAsLeader() {
157     try {
158       synchronized(leaderExists) {
159         if (!leaderExists.get()) {
160           return;
161         }
162         byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
163         if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
164           LOG.info("Stepping down as leader");
165           ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
166           leaderExists.set(false);
167         } else {
168           LOG.info("Not current leader, no need to step down");
169         }
170       }
171     } catch (KeeperException ke) {
172       watcher.abort("Unhandled zookeeper exception removing leader node", ke);
173       candidate.stop("Unhandled zookeeper exception removing leader node: "
174           + ke.getMessage());
175     }
176   }
177 
178   public boolean hasLeader() {
179     return leaderExists.get();
180   }
181 }