001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.master;
020
021import java.io.IOException;
022import java.util.concurrent.atomic.AtomicBoolean;
023
024import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
025import org.apache.hadoop.hbase.zookeeper.ZKUtil;
026import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
027import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
028import org.apache.yetus.audience.InterfaceAudience;
029import org.apache.hadoop.hbase.Server;
030import org.apache.hadoop.hbase.ServerName;
031import org.apache.hadoop.hbase.ZNodeClearer;
032import org.apache.hadoop.hbase.exceptions.DeserializationException;
033import org.apache.hadoop.hbase.monitoring.MonitoredTask;
034import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
035import org.apache.hadoop.hbase.zookeeper.ZKListener;
036import org.apache.zookeeper.KeeperException;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040/**
041 * Handles everything on master-side related to master election.
042 *
043 * <p>Listens and responds to ZooKeeper notifications on the master znode,
044 * both <code>nodeCreated</code> and <code>nodeDeleted</code>.
045 *
046 * <p>Contains blocking methods which will hold up backup masters, waiting
047 * for the active master to fail.
048 *
049 * <p>This class is instantiated in the HMaster constructor and the method
050 * #blockUntilBecomingActiveMaster() is called to wait until becoming
051 * the active master of the cluster.
052 */
053@InterfaceAudience.Private
054public class ActiveMasterManager extends ZKListener {
055  private static final Logger LOG = LoggerFactory.getLogger(ActiveMasterManager.class);
056
057  final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
058  final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
059
060  private final ServerName sn;
061  private int infoPort;
062  private final Server master;
063
064  /**
065   * @param watcher
066   * @param sn ServerName
067   * @param master In an instance of a Master.
068   */
069  ActiveMasterManager(ZKWatcher watcher, ServerName sn, Server master) {
070    super(watcher);
071    watcher.registerListener(this);
072    this.sn = sn;
073    this.master = master;
074  }
075
076  // will be set after jetty server is started
077  public void setInfoPort(int infoPort) {
078    this.infoPort = infoPort;
079  }
080
081  @Override
082  public void nodeCreated(String path) {
083    handle(path);
084  }
085
086  @Override
087  public void nodeDeleted(String path) {
088
089    // We need to keep track of the cluster's shutdown status while
090    // we wait on the current master. We consider that, if the cluster
091    // was already in a "shutdown" state when we started, that this master
092    // is part of a new cluster that was started shortly after the old cluster
093    // shut down, so that state is now irrelevant. This means that the shutdown
094    // state must be set while we wait on the active master in order
095    // to shutdown this master. See HBASE-8519.
096    if(path.equals(watcher.getZNodePaths().clusterStateZNode) && !master.isStopped()) {
097      clusterShutDown.set(true);
098    }
099
100    handle(path);
101  }
102
103  void handle(final String path) {
104    if (path.equals(watcher.getZNodePaths().masterAddressZNode) && !master.isStopped()) {
105      handleMasterNodeChange();
106    }
107  }
108
109  /**
110   * Handle a change in the master node.  Doesn't matter whether this was called
111   * from a nodeCreated or nodeDeleted event because there are no guarantees
112   * that the current state of the master node matches the event at the time of
113   * our next ZK request.
114   *
115   * <p>Uses the watchAndCheckExists method which watches the master address node
116   * regardless of whether it exists or not.  If it does exist (there is an
117   * active master), it returns true.  Otherwise it returns false.
118   *
119   * <p>A watcher is set which guarantees that this method will get called again if
120   * there is another change in the master node.
121   */
122  private void handleMasterNodeChange() {
123    // Watch the node and check if it exists.
124    try {
125      synchronized(clusterHasActiveMaster) {
126        if (ZKUtil.watchAndCheckExists(watcher, watcher.getZNodePaths().masterAddressZNode)) {
127          // A master node exists, there is an active master
128          LOG.trace("A master is now available");
129          clusterHasActiveMaster.set(true);
130        } else {
131          // Node is no longer there, cluster does not have an active master
132          LOG.debug("No master available. Notifying waiting threads");
133          clusterHasActiveMaster.set(false);
134          // Notify any thread waiting to become the active master
135          clusterHasActiveMaster.notifyAll();
136        }
137      }
138    } catch (KeeperException ke) {
139      master.abort("Received an unexpected KeeperException, aborting", ke);
140    }
141  }
142
143  /**
144   * Block until becoming the active master.
145   *
146   * Method blocks until there is not another active master and our attempt
147   * to become the new active master is successful.
148   *
149   * This also makes sure that we are watching the master znode so will be
150   * notified if another master dies.
151   * @param checkInterval the interval to check if the master is stopped
152   * @param startupStatus the monitor status to track the progress
153   * @return True if no issue becoming active master else false if another
154   * master was running or if some other problem (zookeeper, stop flag has been
155   * set on this Master)
156   */
157  boolean blockUntilBecomingActiveMaster(
158      int checkInterval, MonitoredTask startupStatus) {
159    String backupZNode = ZNodePaths.joinZNode(
160      this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString());
161    while (!(master.isAborted() || master.isStopped())) {
162      startupStatus.setStatus("Trying to register in ZK as active master");
163      // Try to become the active master, watch if there is another master.
164      // Write out our ServerName as versioned bytes.
165      try {
166        if (MasterAddressTracker.setMasterAddress(this.watcher,
167            this.watcher.getZNodePaths().masterAddressZNode, this.sn, infoPort)) {
168
169          // If we were a backup master before, delete our ZNode from the backup
170          // master directory since we are the active now)
171          if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
172            LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
173            ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
174          }
175          // Save the znode in a file, this will allow to check if we crash in the launch scripts
176          ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
177
178          // We are the master, return
179          startupStatus.setStatus("Successfully registered as active master.");
180          this.clusterHasActiveMaster.set(true);
181          LOG.info("Registered as active master=" + this.sn);
182          return true;
183        }
184
185        // There is another active master running elsewhere or this is a restart
186        // and the master ephemeral node has not expired yet.
187        this.clusterHasActiveMaster.set(true);
188
189        String msg;
190        byte[] bytes =
191          ZKUtil.getDataAndWatch(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);
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 = ProtobufUtil.parseServerNameFrom(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.isSameAddress(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.getZNodePaths().masterAddressZNode);
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 (clusterHasActiveMaster.get() && !master.isStopped()) {
226          try {
227            clusterHasActiveMaster.wait(checkInterval);
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      }
239    }
240    return false;
241  }
242
243  /**
244   * @return True if cluster has an active master.
245   */
246  boolean hasActiveMaster() {
247    try {
248      if (ZKUtil.checkExists(watcher, watcher.getZNodePaths().masterAddressZNode) >= 0) {
249        return true;
250      }
251    }
252    catch (KeeperException ke) {
253      LOG.info("Received an unexpected KeeperException when checking " +
254          "isActiveMaster : "+ ke);
255    }
256    return false;
257  }
258
259  public void stop() {
260    try {
261      synchronized (clusterHasActiveMaster) {
262        // Master is already stopped, wake up the manager
263        // thread so that it can shutdown soon.
264        clusterHasActiveMaster.notifyAll();
265      }
266      // If our address is in ZK, delete it on our way out
267      ServerName activeMaster = null;
268      try {
269        activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
270      } catch (IOException e) {
271        LOG.warn("Failed get of master address: " + e.toString());
272      }
273      if (activeMaster != null &&  activeMaster.equals(this.sn)) {
274        ZKUtil.deleteNode(watcher, watcher.getZNodePaths().masterAddressZNode);
275        // We may have failed to delete the znode at the previous step, but
276        //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
277        ZNodeClearer.deleteMyEphemeralNodeOnDisk();
278      }
279    } catch (KeeperException e) {
280      LOG.debug(this.watcher.prefix("Failed delete of our master address node; " +
281          e.getMessage()));
282    }
283  }
284}