View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master;
20  
21  import java.io.IOException;
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.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.ZNodeClearer;
28  import org.apache.hadoop.hbase.exceptions.DeserializationException;
29  import org.apache.hadoop.hbase.Server;
30  import org.apache.hadoop.hbase.ServerName;
31  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
32  import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
33  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
34  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
35  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
36  import org.apache.zookeeper.KeeperException;
37  
38  /**
39   * Handles everything on master-side related to master election.
40   *
41   * <p>Listens and responds to ZooKeeper notifications on the master znode,
42   * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
43   *
44   * <p>Contains blocking methods which will hold up backup masters, waiting
45   * for the active master to fail.
46   *
47   * <p>This class is instantiated in the HMaster constructor and the method
48   * #blockUntilBecomingActiveMaster() is called to wait until becoming
49   * the active master of the cluster.
50   */
51  @InterfaceAudience.Private
52  public class ActiveMasterManager extends ZooKeeperListener {
53    private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
54  
55    final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
56    final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
57  
58    private final ServerName sn;
59    private final Server master;
60  
61    /**
62     * @param watcher
63     * @param sn ServerName
64     * @param master In an instance of a Master.
65     */
66    ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
67      super(watcher);
68      this.sn = sn;
69      this.master = master;
70    }
71  
72    @Override
73    public void nodeCreated(String path) {
74      handle(path);
75    }
76  
77    @Override
78    public void nodeDeleted(String path) {
79  
80      // We need to keep track of the cluster's shutdown status while
81      // we wait on the current master. We consider that, if the cluster
82      // was already in a "shutdown" state when we started, that this master
83      // is part of a new cluster that was started shortly after the old cluster
84      // shut down, so that state is now irrelevant. This means that the shutdown
85      // state must be set while we wait on the active master in order
86      // to shutdown this master. See HBASE-8519.
87      if(path.equals(watcher.clusterStateZNode) && !master.isStopped()) {
88        clusterShutDown.set(true);
89      }
90  
91      handle(path);
92    }
93  
94    void handle(final String path) {
95      if (path.equals(watcher.getMasterAddressZNode()) && !master.isStopped()) {
96        handleMasterNodeChange();
97      }
98    }
99  
100   /**
101    * Handle a change in the master node.  Doesn't matter whether this was called
102    * from a nodeCreated or nodeDeleted event because there are no guarantees
103    * that the current state of the master node matches the event at the time of
104    * our next ZK request.
105    *
106    * <p>Uses the watchAndCheckExists method which watches the master address node
107    * regardless of whether it exists or not.  If it does exist (there is an
108    * active master), it returns true.  Otherwise it returns false.
109    *
110    * <p>A watcher is set which guarantees that this method will get called again if
111    * there is another change in the master node.
112    */
113   private void handleMasterNodeChange() {
114     // Watch the node and check if it exists.
115     try {
116       synchronized(clusterHasActiveMaster) {
117         if (ZKUtil.watchAndCheckExists(watcher, watcher.getMasterAddressZNode())) {
118           // A master node exists, there is an active master
119           LOG.debug("A master is now available");
120           clusterHasActiveMaster.set(true);
121         } else {
122           // Node is no longer there, cluster does not have an active master
123           LOG.debug("No master available. Notifying waiting threads");
124           clusterHasActiveMaster.set(false);
125           // Notify any thread waiting to become the active master
126           clusterHasActiveMaster.notifyAll();
127         }
128       }
129     } catch (KeeperException ke) {
130       master.abort("Received an unexpected KeeperException, aborting", ke);
131     }
132   }
133 
134   /**
135    * Block until becoming the active master.
136    *
137    * Method blocks until there is not another active master and our attempt
138    * to become the new active master is successful.
139    *
140    * This also makes sure that we are watching the master znode so will be
141    * notified if another master dies.
142    * @param startupStatus
143    * @return True if no issue becoming active master else false if another
144    * master was running or if some other problem (zookeeper, stop flag has been
145    * set on this Master)
146    */
147   boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus) {
148     while (true) {
149       startupStatus.setStatus("Trying to register in ZK as active master");
150       // Try to become the active master, watch if there is another master.
151       // Write out our ServerName as versioned bytes.
152       try {
153         String backupZNode =
154             ZKUtil.joinZNode(this.watcher.backupMasterAddressesZNode, this.sn.toString());
155         if (MasterAddressTracker.setMasterAddress(this.watcher,
156             this.watcher.getMasterAddressZNode(), this.sn)) {
157 
158           // If we were a backup master before, delete our ZNode from the backup
159           // master directory since we are the active now)
160           if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
161             LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
162             ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
163           }
164           // Save the znode in a file, this will allow to check if we crash in the launch scripts
165           ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
166 
167           // We are the master, return
168           startupStatus.setStatus("Successfully registered as active master.");
169           this.clusterHasActiveMaster.set(true);
170           LOG.info("Registered Active Master=" + this.sn);
171           return true;
172         }
173 
174         // There is another active master running elsewhere or this is a restart
175         // and the master ephemeral node has not expired yet.
176         this.clusterHasActiveMaster.set(true);
177 
178         /*
179         * Add a ZNode for ourselves in the backup master directory since we are
180         * not the active master.
181         *
182         * If we become the active master later, ActiveMasterManager will delete
183         * this node explicitly.  If we crash before then, ZooKeeper will delete
184         * this node for us since it is ephemeral.
185         */
186         LOG.info("Adding ZNode for " + backupZNode + " in backup master directory");
187         MasterAddressTracker.setMasterAddress(this.watcher, backupZNode, this.sn);
188 
189         String msg;
190         byte[] bytes =
191           ZKUtil.getDataAndWatch(this.watcher, this.watcher.getMasterAddressZNode());
192         if (bytes == null) {
193           msg = ("A master was detected, but went down before its address " +
194             "could be read.  Attempting to become the next active master");
195         } else {
196           ServerName currentMaster;
197           try {
198             currentMaster = ServerName.parseFrom(bytes);
199           } catch (DeserializationException e) {
200             LOG.warn("Failed parse", e);
201             // Hopefully next time around we won't fail the parse.  Dangerous.
202             continue;
203           }
204           if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
205             msg = ("Current master has this master's address, " +
206               currentMaster + "; master was restarted? Deleting node.");
207             // Hurry along the expiration of the znode.
208             ZKUtil.deleteNode(this.watcher, this.watcher.getMasterAddressZNode());
209 
210             // We may have failed to delete the znode at the previous step, but
211             //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
212             ZNodeClearer.deleteMyEphemeralNodeOnDisk();
213           } else {
214             msg = "Another master is the active master, " + currentMaster +
215               "; waiting to become the next active master";
216           }
217         }
218         LOG.info(msg);
219         startupStatus.setStatus(msg);
220       } catch (KeeperException ke) {
221         master.abort("Received an unexpected KeeperException, aborting", ke);
222         return false;
223       }
224       synchronized (this.clusterHasActiveMaster) {
225         while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
226           try {
227             this.clusterHasActiveMaster.wait();
228           } catch (InterruptedException e) {
229             // We expect to be interrupted when a master dies,
230             //  will fall out if so
231             LOG.debug("Interrupted waiting for master to die", e);
232           }
233         }
234         if (clusterShutDown.get()) {
235           this.master.stop(
236             "Cluster went down before this master became active");
237         }
238         if (this.master.isStopped()) {
239           return false;
240         }
241         // there is no active master so we can try to become active master again
242       }
243     }
244   }
245 
246   /**
247    * @return True if cluster has an active master.
248    */
249   public boolean isActiveMaster() {
250     try {
251       if (ZKUtil.checkExists(watcher, watcher.getMasterAddressZNode()) >= 0) {
252         return true;
253       }
254     }
255     catch (KeeperException ke) {
256       LOG.info("Received an unexpected KeeperException when checking " +
257           "isActiveMaster : "+ ke);
258     }
259     return false;
260   }
261 
262   public void stop() {
263     try {
264       // If our address is in ZK, delete it on our way out
265       ServerName activeMaster = null;
266       try {
267         activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
268       } catch (IOException e) {
269         LOG.warn("Failed get of master address: " + e.toString());
270       }
271       if (activeMaster != null &&  activeMaster.equals(this.sn)) {
272         ZKUtil.deleteNode(watcher, watcher.getMasterAddressZNode());
273         // We may have failed to delete the znode at the previous step, but
274         //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
275         ZNodeClearer.deleteMyEphemeralNodeOnDisk();
276       }
277     } catch (KeeperException e) {
278       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
279     }
280   }
281 }