View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import java.util.concurrent.atomic.AtomicBoolean;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.HServerAddress;
27  import org.apache.hadoop.hbase.Server;
28  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
29  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
30  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31  import org.apache.zookeeper.KeeperException;
32  
33  /**
34   * Handles everything on master-side related to master election.
35   *
36   * <p>Listens and responds to ZooKeeper notifications on the master znode,
37   * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
38   *
39   * <p>Contains blocking methods which will hold up backup masters, waiting
40   * for the active master to fail.
41   *
42   * <p>This class is instantiated in the HMaster constructor and the method
43   * {@link #blockUntilBecomingActiveMaster()} is called to wait until becoming
44   * the active master of the cluster.
45   */
46  class ActiveMasterManager extends ZooKeeperListener {
47    private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
48  
49    final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
50  
51    private final HServerAddress address;
52    private final Server master;
53  
54    ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address,
55        Server master) {
56      super(watcher);
57      this.address = address;
58      this.master = master;
59    }
60  
61    @Override
62    public void nodeCreated(String path) {
63      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
64        handleMasterNodeChange();
65      }
66    }
67  
68    @Override
69    public void nodeDeleted(String path) {
70      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
71        handleMasterNodeChange();
72      }
73    }
74  
75    /**
76     * Handle a change in the master node.  Doesn't matter whether this was called
77     * from a nodeCreated or nodeDeleted event because there are no guarantees
78     * that the current state of the master node matches the event at the time of
79     * our next ZK request.
80     *
81     * <p>Uses the watchAndCheckExists method which watches the master address node
82     * regardless of whether it exists or not.  If it does exist (there is an
83     * active master), it returns true.  Otherwise it returns false.
84     *
85     * <p>A watcher is set which guarantees that this method will get called again if
86     * there is another change in the master node.
87     */
88    private void handleMasterNodeChange() {
89      // Watch the node and check if it exists.
90      try {
91        synchronized(clusterHasActiveMaster) {
92          if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
93            // A master node exists, there is an active master
94            LOG.debug("A master is now available");
95            clusterHasActiveMaster.set(true);
96          } else {
97            // Node is no longer there, cluster does not have an active master
98            LOG.debug("No master available. Notifying waiting threads");
99            clusterHasActiveMaster.set(false);
100           // Notify any thread waiting to become the active master
101           clusterHasActiveMaster.notifyAll();
102         }
103       }
104     } catch (KeeperException ke) {
105       master.abort("Received an unexpected KeeperException, aborting", ke);
106     }
107   }
108 
109   /**
110    * Block until becoming the active master.
111    *
112    * Method blocks until there is not another active master and our attempt
113    * to become the new active master is successful.
114    *
115    * This also makes sure that we are watching the master znode so will be
116    * notified if another master dies.
117    * @return True if no issue becoming active master else false if another
118    * master was running or if some other problem (zookeeper, stop flag has been
119    * set on this Master)
120    */
121   boolean blockUntilBecomingActiveMaster() {
122     boolean cleanSetOfActiveMaster = true;
123     // Try to become the active master, watch if there is another master
124     try {
125       if (ZKUtil.setAddressAndWatch(this.watcher,
126           this.watcher.masterAddressZNode, this.address)) {
127         // We are the master, return
128         this.clusterHasActiveMaster.set(true);
129         LOG.info("Master=" + this.address);
130         return cleanSetOfActiveMaster;
131       }
132       cleanSetOfActiveMaster = false;
133 
134       // There is another active master running elsewhere or this is a restart
135       // and the master ephemeral node has not expired yet.
136       this.clusterHasActiveMaster.set(true);
137       HServerAddress currentMaster =
138         ZKUtil.getDataAsAddress(this.watcher, this.watcher.masterAddressZNode);
139       if (currentMaster != null && currentMaster.equals(this.address)) {
140         LOG.info("Current master has this master's address, " + currentMaster +
141           "; master was restarted?  Waiting on znode to expire...");
142         // Hurry along the expiration of the znode.
143         ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
144       } else {
145         LOG.info("Another master is the active master, " + currentMaster +
146           "; waiting to become the next active master");
147       }
148     } catch (KeeperException ke) {
149       master.abort("Received an unexpected KeeperException, aborting", ke);
150       return false;
151     }
152     synchronized (this.clusterHasActiveMaster) {
153       while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
154         try {
155           this.clusterHasActiveMaster.wait();
156         } catch (InterruptedException e) {
157           // We expect to be interrupted when a master dies, will fall out if so
158           LOG.debug("Interrupted waiting for master to die", e);
159         }
160       }
161       if (this.master.isStopped()) {
162         return cleanSetOfActiveMaster;
163       }
164       // Try to become active master again now that there is no active master
165       blockUntilBecomingActiveMaster();
166     }
167     return cleanSetOfActiveMaster;
168   }
169 
170   /**
171    * @return True if cluster has an active master.
172    */
173   public boolean isActiveMaster() {
174     try {
175       if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
176         return true;
177       }
178     } 
179     catch (KeeperException ke) {
180       LOG.info("Received an unexpected KeeperException when checking " +
181           "isActiveMaster : "+ ke);
182     }
183     return false;
184   }
185 
186   public void stop() {
187     try {
188       // If our address is in ZK, delete it on our way out
189       HServerAddress zkAddress =
190         ZKUtil.getDataAsAddress(watcher, watcher.masterAddressZNode);
191       // TODO: redo this to make it atomic (only added for tests)
192       if(zkAddress != null &&
193           zkAddress.equals(address)) {
194         ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
195       }
196     } catch (KeeperException e) {
197       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
198     }
199   }
200 }