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