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