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  import com.google.common.collect.ImmutableList;
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.util.List;
24  import java.util.concurrent.atomic.AtomicBoolean;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.Server;
30  import org.apache.hadoop.hbase.ServerName;
31  import org.apache.hadoop.hbase.ZNodeClearer;
32  import org.apache.hadoop.hbase.exceptions.DeserializationException;
33  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
34  import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
35  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
36  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
37  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
38  import org.apache.zookeeper.KeeperException;
39  
40  /**
41   * Handles everything on master-side related to master election. Keeps track of
42   * currently active master and registered backup masters.
43   *
44   * <p>Listens and responds to ZooKeeper notifications on the master znodes,
45   * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
46   *
47   * <p>Contains blocking methods which will hold up backup masters, waiting
48   * for the active master to fail.
49   *
50   * <p>This class is instantiated in the HMaster constructor and the method
51   * #blockUntilBecomingActiveMaster() is called to wait until becoming
52   * the active master of the cluster.
53   */
54  @InterfaceAudience.Private
55  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
56    justification="Use of an atomic type both as monitor and condition variable is intended")
57  public class ActiveMasterManager extends ZooKeeperListener {
58    private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
59  
60    final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
61    final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
62  
63    // This server's information.
64    private final ServerName sn;
65    private int infoPort;
66    private final Server master;
67  
68    // Active master's server name. Invalidated anytime active master changes (based on ZK
69    // notifications) and lazily fetched on-demand.
70    // ServerName is immutable, so we don't need heavy synchronization around it.
71    volatile ServerName activeMasterServerName;
72    // Registered backup masters. List is kept up to date based on ZK change notifications to
73    // backup znode.
74    private volatile ImmutableList<ServerName> backupMasters;
75  
76    /**
77     * @param watcher ZK watcher
78     * @param sn ServerName
79     * @param master In an instance of a Master.
80     */
81    ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master)
82        throws InterruptedIOException {
83      super(watcher);
84      watcher.registerListener(this);
85      this.sn = sn;
86      this.master = master;
87      updateBackupMasters();
88    }
89  
90    // will be set after jetty server is started
91    public void setInfoPort(int infoPort) {
92      this.infoPort = infoPort;
93    }
94  
95    @Override
96    public void nodeCreated(String path) {
97      handle(path);
98    }
99  
100   @Override
101   public void nodeChildrenChanged(String path) {
102     if (path.equals(watcher.backupMasterAddressesZNode)) {
103       try {
104         updateBackupMasters();
105       } catch (InterruptedIOException ioe) {
106         LOG.error("Error updating backup masters", ioe);
107       }
108     }
109   }
110 
111   @Override
112   public void nodeDeleted(String path) {
113     // We need to keep track of the cluster's shutdown status while
114     // we wait on the current master. We consider that, if the cluster
115     // was already in a "shutdown" state when we started, that this master
116     // is part of a new cluster that was started shortly after the old cluster
117     // shut down, so that state is now irrelevant. This means that the shutdown
118     // state must be set while we wait on the active master in order
119     // to shutdown this master. See HBASE-8519.
120     if(path.equals(watcher.clusterStateZNode) && !master.isStopped()) {
121       clusterShutDown.set(true);
122     }
123     handle(path);
124   }
125 
126   void handle(final String path) {
127     if (path.equals(watcher.getMasterAddressZNode()) && !master.isStopped()) {
128       handleMasterNodeChange();
129     }
130   }
131 
132   private void updateBackupMasters() throws InterruptedIOException {
133     backupMasters =
134         ImmutableList.copyOf(MasterAddressTracker.getBackupMastersAndRenewWatch(watcher));
135   }
136 
137   /**
138    * Fetches the active master's ServerName from zookeeper.
139    */
140   private void fetchAndSetActiveMasterServerName() {
141     LOG.debug("Attempting to fetch active master sn from zk");
142     try {
143       activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
144     } catch (IOException | KeeperException e) {
145       // Log and ignore for now and re-fetch later if needed.
146       LOG.error("Error fetching active master information", e);
147     }
148   }
149 
150   /**
151    * @return the currently active master as seen by us or null if one does not exist.
152    */
153   public ServerName getActiveMasterServerName() {
154     if (!clusterHasActiveMaster.get()) {
155       return null;
156     }
157     if (activeMasterServerName == null) {
158       fetchAndSetActiveMasterServerName();
159     }
160     // It could still be null, but return whatever we have.
161     return activeMasterServerName;
162   }
163 
164   /**
165    * Handle a change in the master node.  Doesn't matter whether this was called
166    * from a nodeCreated or nodeDeleted event because there are no guarantees
167    * that the current state of the master node matches the event at the time of
168    * our next ZK request.
169    *
170    * <p>Uses the watchAndCheckExists method which watches the master address node
171    * regardless of whether it exists or not.  If it does exist (there is an
172    * active master), it returns true.  Otherwise it returns false.
173    *
174    * <p>A watcher is set which guarantees that this method will get called again if
175    * there is another change in the master node.
176    */
177   private void handleMasterNodeChange() {
178     // Watch the node and check if it exists.
179     try {
180       synchronized(clusterHasActiveMaster) {
181         if (ZKUtil.watchAndCheckExists(watcher, watcher.getMasterAddressZNode())) {
182           // A master node exists, there is an active master
183           LOG.debug("A master is now available");
184           clusterHasActiveMaster.set(true);
185         } else {
186           // Node is no longer there, cluster does not have an active master
187           LOG.debug("No master available. Notifying waiting threads");
188           clusterHasActiveMaster.set(false);
189           // Notify any thread waiting to become the active master
190           clusterHasActiveMaster.notifyAll();
191         }
192         // Reset the active master sn. Will be re-fetched later if needed.
193         // We don't want to make a synchronous RPC under a monitor.
194         activeMasterServerName = null;
195       }
196     } catch (KeeperException ke) {
197       master.abort("Received an unexpected KeeperException, aborting", ke);
198     }
199   }
200 
201   /**
202    * Block until becoming the active master.
203    *
204    * Method blocks until there is not another active master and our attempt
205    * to become the new active master is successful.
206    *
207    * This also makes sure that we are watching the master znode so will be
208    * notified if another master dies.
209    * @param checkInterval the interval to check if the master is stopped
210    * @param startupStatus the monitor status to track the progress
211    * @return True if no issue becoming active master else false if another
212    *   master was running or if some other problem (zookeeper, stop flag has been
213    *   set on this Master)
214    */
215   boolean blockUntilBecomingActiveMaster(
216       int checkInterval, MonitoredTask startupStatus) {
217     String backupZNode = ZKUtil.joinZNode(
218       this.watcher.backupMasterAddressesZNode, this.sn.toString());
219     while (!(master.isAborted() || master.isStopped())) {
220       startupStatus.setStatus("Trying to register in ZK as active master");
221       // Try to become the active master, watch if there is another master.
222       // Write out our ServerName as versioned bytes.
223       try {
224         if (MasterAddressTracker.setMasterAddress(this.watcher,
225             this.watcher.getMasterAddressZNode(), this.sn, infoPort)) {
226 
227           // If we were a backup master before, delete our ZNode from the backup
228           // master directory since we are the active now)
229           if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
230             LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
231             ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
232           }
233           // Save the znode in a file, this will allow to check if we crash in the launch scripts
234           ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
235 
236           // We are the master, return
237           startupStatus.setStatus("Successfully registered as active master.");
238           this.clusterHasActiveMaster.set(true);
239           LOG.info("Registered Active Master=" + this.sn);
240           activeMasterServerName = sn;
241           return true;
242         }
243 
244         // Invalidate the active master name so that subsequent requests do not get any stale
245         // master information. Will be re-fetched if needed.
246         activeMasterServerName = null;
247         // There is another active master running elsewhere or this is a restart
248         // and the master ephemeral node has not expired yet.
249         this.clusterHasActiveMaster.set(true);
250 
251         String msg;
252         byte[] bytes =
253           ZKUtil.getDataAndWatch(this.watcher, this.watcher.getMasterAddressZNode());
254         if (bytes == null) {
255           msg = ("A master was detected, but went down before its address " +
256             "could be read.  Attempting to become the next active master");
257         } else {
258           ServerName currentMaster;
259           try {
260             currentMaster = ServerName.parseFrom(bytes);
261           } catch (DeserializationException e) {
262             LOG.warn("Failed parse", e);
263             // Hopefully next time around we won't fail the parse.  Dangerous.
264             continue;
265           }
266           if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
267             msg = ("Current master has this master's address, " +
268               currentMaster + "; master was restarted? Deleting node.");
269             // Hurry along the expiration of the znode.
270             ZKUtil.deleteNode(this.watcher, this.watcher.getMasterAddressZNode());
271 
272             // We may have failed to delete the znode at the previous step, but
273             //  we delete the file anyway: a second attempt to delete the znode is likely to fail
274             //  again.
275             ZNodeClearer.deleteMyEphemeralNodeOnDisk();
276           } else {
277             msg = "Another master is the active master, " + currentMaster +
278               "; waiting to become the next active master";
279           }
280         }
281         LOG.info(msg);
282         startupStatus.setStatus(msg);
283       } catch (KeeperException ke) {
284         master.abort("Received an unexpected KeeperException, aborting", ke);
285         return false;
286       }
287       synchronized (this.clusterHasActiveMaster) {
288         while (clusterHasActiveMaster.get() && !master.isStopped()) {
289           try {
290             clusterHasActiveMaster.wait(checkInterval);
291           } catch (InterruptedException e) {
292             // We expect to be interrupted when a master dies,
293             //  will fall out if so
294             LOG.debug("Interrupted waiting for master to die", e);
295           }
296         }
297         if (clusterShutDown.get()) {
298           this.master.stop(
299             "Cluster went down before this master became active");
300         }
301       }
302     }
303     return false;
304   }
305 
306   /**
307    * @return True if cluster has an active master.
308    */
309   boolean hasActiveMaster() {
310     try {
311       if (ZKUtil.checkExists(watcher, watcher.getMasterAddressZNode()) >= 0) {
312         return true;
313       }
314     }
315     catch (KeeperException ke) {
316       LOG.info("Received an unexpected KeeperException when checking " +
317           "isActiveMaster : "+ ke);
318     }
319     return false;
320   }
321 
322   public void stop() {
323     try {
324       synchronized (clusterHasActiveMaster) {
325         // Master is already stopped, wake up the manager
326         // thread so that it can shutdown soon.
327         clusterHasActiveMaster.notifyAll();
328       }
329       // If our address is in ZK, delete it on our way out
330       ServerName activeMaster = null;
331       try {
332         activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
333       } catch (IOException e) {
334         LOG.warn("Failed get of master address: " + e.toString());
335       }
336       if (activeMaster != null &&  activeMaster.equals(this.sn)) {
337         ZKUtil.deleteNode(watcher, watcher.getMasterAddressZNode());
338         // We may have failed to delete the znode at the previous step, but
339         //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
340         ZNodeClearer.deleteMyEphemeralNodeOnDisk();
341       }
342     } catch (KeeperException e) {
343       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
344     }
345   }
346 
347   /**
348    * @return list of registered backup masters.
349    */
350   public List<ServerName> getBackupMasters() {
351     return backupMasters;
352   }
353 }