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.util.Bytes;
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  class ActiveMasterManager extends ZooKeeperListener {
49    private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
50  
51    final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
52  
53    private final ServerName sn;
54    private final Server master;
55  
56    /**
57     * @param watcher
58     * @param sn ServerName
59     * @param master In an instance of a Master.
60     */
61    ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
62      super(watcher);
63      this.sn = sn;
64      this.master = master;
65    }
66  
67    @Override
68    public void nodeCreated(String path) {
69      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
70        handleMasterNodeChange();
71      }
72    }
73  
74    @Override
75    public void nodeDeleted(String path) {
76      if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
77        handleMasterNodeChange();
78      }
79    }
80  
81    /**
82     * Handle a change in the master node.  Doesn't matter whether this was called
83     * from a nodeCreated or nodeDeleted event because there are no guarantees
84     * that the current state of the master node matches the event at the time of
85     * our next ZK request.
86     *
87     * <p>Uses the watchAndCheckExists method which watches the master address node
88     * regardless of whether it exists or not.  If it does exist (there is an
89     * active master), it returns true.  Otherwise it returns false.
90     *
91     * <p>A watcher is set which guarantees that this method will get called again if
92     * there is another change in the master node.
93     */
94    private void handleMasterNodeChange() {
95      // Watch the node and check if it exists.
96      try {
97        synchronized(clusterHasActiveMaster) {
98          if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
99            // A master node exists, there is an active master
100           LOG.debug("A master is now available");
101           clusterHasActiveMaster.set(true);
102         } else {
103           // Node is no longer there, cluster does not have an active master
104           LOG.debug("No master available. Notifying waiting threads");
105           clusterHasActiveMaster.set(false);
106           // Notify any thread waiting to become the active master
107           clusterHasActiveMaster.notifyAll();
108         }
109       }
110     } catch (KeeperException ke) {
111       master.abort("Received an unexpected KeeperException, aborting", ke);
112     }
113   }
114 
115   /**
116    * Block until becoming the active master.
117    *
118    * Method blocks until there is not another active master and our attempt
119    * to become the new active master is successful.
120    *
121    * This also makes sure that we are watching the master znode so will be
122    * notified if another master dies.
123    * @param startupStatus 
124    * @return True if no issue becoming active master else false if another
125    * master was running or if some other problem (zookeeper, stop flag has been
126    * set on this Master)
127    */
128   boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus) {
129     startupStatus.setStatus("Trying to register in ZK as active master");
130     boolean cleanSetOfActiveMaster = true;
131     // Try to become the active master, watch if there is another master.
132     // Write out our ServerName as versioned bytes.
133     try {
134       if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
135           this.watcher.masterAddressZNode, sn.getVersionedBytes())) {
136         // We are the master, return
137         startupStatus.setStatus("Successfully registered as active master.");
138         this.clusterHasActiveMaster.set(true);
139         LOG.info("Master=" + this.sn);
140         return cleanSetOfActiveMaster;
141       }
142       cleanSetOfActiveMaster = false;
143 
144       // There is another active master running elsewhere or this is a restart
145       // and the master ephemeral node has not expired yet.
146       this.clusterHasActiveMaster.set(true);
147       byte [] bytes =
148         ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
149       ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
150       if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
151         String msg = ("Current master has this master's address, " + currentMaster +
152           "; master was restarted?  Waiting on znode to expire...");
153         LOG.info(msg);
154         startupStatus.setStatus(msg);
155         // Hurry along the expiration of the znode.
156         ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
157       } else {
158         String msg = "Another master is the active master, " + currentMaster +
159         "; waiting to become the next active master";
160         LOG.info(msg);
161         startupStatus.setStatus(msg);
162       }
163     } catch (KeeperException ke) {
164       master.abort("Received an unexpected KeeperException, aborting", ke);
165       return false;
166     }
167     synchronized (this.clusterHasActiveMaster) {
168       while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
169         try {
170           this.clusterHasActiveMaster.wait();
171         } catch (InterruptedException e) {
172           // We expect to be interrupted when a master dies, will fall out if so
173           LOG.debug("Interrupted waiting for master to die", e);
174         }
175       }
176       if (this.master.isStopped()) {
177         return cleanSetOfActiveMaster;
178       }
179       // Try to become active master again now that there is no active master
180       blockUntilBecomingActiveMaster(startupStatus);
181     }
182     return cleanSetOfActiveMaster;
183   }
184 
185   /**
186    * @return True if cluster has an active master.
187    */
188   public boolean isActiveMaster() {
189     try {
190       if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
191         return true;
192       }
193     } 
194     catch (KeeperException ke) {
195       LOG.info("Received an unexpected KeeperException when checking " +
196           "isActiveMaster : "+ ke);
197     }
198     return false;
199   }
200 
201   public void stop() {
202     try {
203       // If our address is in ZK, delete it on our way out
204       byte [] bytes =
205         ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
206       // TODO: redo this to make it atomic (only added for tests)
207       ServerName master = new ServerName(Bytes.toString(bytes));
208       if (master != null &&  master.equals(this.sn)) {
209         ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
210       }
211     } catch (KeeperException e) {
212       LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
213     }
214   }
215 }