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