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.Server;
27  import org.apache.hadoop.hbase.ServerName;
28  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
29  import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
30  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
31  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
32  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
33  import org.apache.zookeeper.KeeperException;
34  
35  /**
36   * Handles everything on master-side related to master election.
37   *
38   * <p>Listens and responds to ZooKeeper notifications on the master znode,
39   * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
40   *
41   * <p>Contains blocking methods which will hold up backup masters, waiting
42   * for the active master to fail.
43   *
44   * <p>This class is instantiated in the HMaster constructor and the method
45   * #blockUntilBecomingActiveMaster() is called to wait until becoming
46   * the active master of the cluster.
47   */
48  public class ActiveMasterManager extends ZooKeeperListener {
49    private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
50  
51    final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
52  
53    private final ServerName sn;
54    private final Server master;
55  
56    /**
57     * @param watcher
58     * @param sn ServerName
59     * @param master In an instance of a Master.
60     */
61    ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
62      super(watcher);
63      this.sn = sn;
64      this.master = master;
65    }
66  
67    @Override
68    public void nodeCreated(String path) {
69      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
70        handleMasterNodeChange();
71      }
72    }
73  
74    @Override
75    public void nodeDeleted(String path) {
76      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
77        handleMasterNodeChange();
78      }
79    }
80  
81    /**
82     * Handle a change in the master node.  Doesn't matter whether this was called
83     * from a nodeCreated or nodeDeleted event because there are no guarantees
84     * that the current state of the master node matches the event at the time of
85     * our next ZK request.
86     *
87     * <p>Uses the watchAndCheckExists method which watches the master address node
88     * regardless of whether it exists or not.  If it does exist (there is an
89     * active master), it returns true.  Otherwise it returns false.
90     *
91     * <p>A watcher is set which guarantees that this method will get called again if
92     * there is another change in the master node.
93     */
94    private void handleMasterNodeChange() {
95      // Watch the node and check if it exists.
96      try {
97        synchronized(clusterHasActiveMaster) {
98          if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
99            // A master node exists, there is an active master
100           LOG.debug("A master is now available");
101           clusterHasActiveMaster.set(true);
102         } else {
103           // Node is no longer there, cluster does not have an active master
104           LOG.debug("No master available. Notifying waiting threads");
105           clusterHasActiveMaster.set(false);
106           // Notify any thread waiting to become the active master
107           clusterHasActiveMaster.notifyAll();
108         }
109       }
110     } catch (KeeperException ke) {
111       master.abort("Received an unexpected KeeperException, aborting", ke);
112     }
113   }
114 
115   /**
116    * Block until becoming the active master.
117    *
118    * Method blocks until there is not another active master and our attempt
119    * to become the new active master is successful.
120    *
121    * This also makes sure that we are watching the master znode so will be
122    * notified if another master dies.
123    * @param startupStatus
124    * @return True if no issue becoming active master else false if another
125    * master was running or if some other problem (zookeeper, stop flag has been
126    * set on this Master)
127    */
128   boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus,
129     ClusterStatusTracker clusterStatusTracker) {
130     while (true) {
131       startupStatus.setStatus("Trying to register in ZK as active master");
132       // Try to become the active master, watch if there is another master.
133       // Write out our ServerName as versioned bytes.
134       try {
135         String backupZNode = ZKUtil.joinZNode(
136           this.watcher.backupMasterAddressesZNode, this.sn.toString());
137         if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
138           this.watcher.masterAddressZNode, this.sn.getVersionedBytes())) {
139           // If we were a backup master before, delete our ZNode from the backup
140           // master directory since we are the active now
141           LOG.info("Deleting ZNode for " + backupZNode +
142             " from backup master directory");
143           ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
144 
145           // We are the master, return
146           startupStatus.setStatus("Successfully registered as active master.");
147           this.clusterHasActiveMaster.set(true);
148           LOG.info("Master=" + this.sn);
149           return true;
150         }
151 
152         // There is another active master running elsewhere or this is a restart
153         // and the master ephemeral node has not expired yet.
154         this.clusterHasActiveMaster.set(true);
155 
156         /*
157          * Add a ZNode for ourselves in the backup master directory since we are
158          * not the active master.
159          *
160          * If we become the active master later, ActiveMasterManager will delete
161          * this node explicitly.  If we crash before then, ZooKeeper will delete
162          * this node for us since it is ephemeral.
163          */
164         LOG.info("Adding ZNode for " + backupZNode +
165           " in backup master directory");
166         ZKUtil.createEphemeralNodeAndWatch(this.watcher, backupZNode,
167           this.sn.getVersionedBytes());
168 
169         String msg;
170         byte [] bytes =
171           ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
172         if (bytes == null) {
173           msg = ("A master was detected, but went down before its address " +
174             "could be read.  Attempting to become the next active master");
175         } else {
176           ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
177           if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
178             msg = ("Current master has this master's address, " +
179               currentMaster + "; master was restarted? Deleting node.");
180             // Hurry along the expiration of the znode.
181             ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
182           } else {
183             msg = "Another master is the active master, " + currentMaster +
184               "; waiting to become the next active master";
185           }
186         }
187         LOG.info(msg);
188         startupStatus.setStatus(msg);
189       } catch (KeeperException ke) {
190         master.abort("Received an unexpected KeeperException, aborting", ke);
191         return false;
192       }
193       synchronized (this.clusterHasActiveMaster) {
194         while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
195           try {
196             this.clusterHasActiveMaster.wait();
197           } catch (InterruptedException e) {
198             // We expect to be interrupted when a master dies, will fall out if so
199             LOG.debug("Interrupted waiting for master to die", e);
200           }
201         }
202         if (!clusterStatusTracker.isClusterUp()) {
203           this.master.stop("Cluster went down before this master became active");
204         }
205         if (this.master.isStopped()) {
206           return false;
207         }
208         // Try to become active master again now that there is no active master
209       }
210     }
211   }
212 
213   /**
214    * @return True if cluster has an active master.
215    */
216   public boolean isActiveMaster() {
217     try {
218       if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
219         return true;
220       }
221     } 
222     catch (KeeperException ke) {
223       LOG.info("Received an unexpected KeeperException when checking " +
224           "isActiveMaster : "+ ke);
225     }
226     return false;
227   }
228 
229   public void stop() {
230     try {
231       // If our address is in ZK, delete it on our way out
232       byte [] bytes =
233         ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
234       // TODO: redo this to make it atomic (only added for tests)
235       ServerName master = bytes == null ? null : ServerName.parseVersionedServerName(bytes);
236       if (master != null &&  master.equals(this.sn)) {
237         ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
238       }
239     } catch (KeeperException e) {
240       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
241     }
242   }
243 
244   /**
245    * @return the ServerName for the current active master
246    */
247   public ServerName getActiveMaster() {
248     ServerName sn = null;
249     String msg;
250     try {
251       byte[] bytes = ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
252       if (bytes == null) {
253         msg = "A master was detected, but went down before its address.";
254         LOG.info(msg);
255       } else {
256         sn = ServerName.parseVersionedServerName(bytes);
257       }
258     } catch (KeeperException e) {
259       msg = "Could not find active master";
260       LOG.info(msg);
261     }
262     return sn;
263   }
264 }