001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.master;
019
020import java.io.IOException;
021import java.io.InterruptedIOException;
022import java.util.List;
023import java.util.Optional;
024import java.util.concurrent.atomic.AtomicBoolean;
025import org.apache.hadoop.hbase.Server;
026import org.apache.hadoop.hbase.ServerName;
027import org.apache.hadoop.hbase.ZNodeClearer;
028import org.apache.hadoop.hbase.exceptions.DeserializationException;
029import org.apache.hadoop.hbase.monitoring.MonitoredTask;
030import org.apache.hadoop.hbase.monitoring.TaskGroup;
031import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
032import org.apache.hadoop.hbase.zookeeper.ZKListener;
033import org.apache.hadoop.hbase.zookeeper.ZKUtil;
034import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
035import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.apache.zookeeper.KeeperException;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
042
043import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
044
045/**
046 * Handles everything on master-side related to master election. Keeps track of currently active
047 * master and registered backup masters.
048 * <p>
049 * Listens and responds to ZooKeeper notifications on the master znodes, both
050 * <code>nodeCreated</code> and <code>nodeDeleted</code>.
051 * <p>
052 * Contains blocking methods which will hold up backup masters, waiting for the active master to
053 * fail.
054 * <p>
055 * This class is instantiated in the HMaster constructor and the method
056 * #blockUntilBecomingActiveMaster() is called to wait until becoming the active master of the
057 * cluster.
058 */
059@InterfaceAudience.Private
060public class ActiveMasterManager extends ZKListener {
061  private static final Logger LOG = LoggerFactory.getLogger(ActiveMasterManager.class);
062
063  final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
064  final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
065
066  // This server's information. Package-private for child implementations.
067  int infoPort;
068  final ServerName sn;
069  final Server master;
070
071  // Active master's server name. Invalidated anytime active master changes (based on ZK
072  // notifications) and lazily fetched on-demand.
073  // ServerName is immutable, so we don't need heavy synchronization around it.
074  volatile ServerName activeMasterServerName;
075  // Registered backup masters. List is kept up to date based on ZK change notifications to
076  // backup znode.
077  private volatile ImmutableList<ServerName> backupMasters;
078
079  /**
080   * @param watcher ZK watcher
081   * @param sn      ServerName
082   * @param master  In an instance of a Master.
083   */
084  ActiveMasterManager(ZKWatcher watcher, ServerName sn, Server master)
085    throws InterruptedIOException {
086    super(watcher);
087    watcher.registerListener(this);
088    this.sn = sn;
089    this.master = master;
090    updateBackupMasters();
091  }
092
093  // will be set after jetty server is started
094  public void setInfoPort(int infoPort) {
095    this.infoPort = infoPort;
096  }
097
098  @Override
099  public void nodeCreated(String path) {
100    handle(path);
101  }
102
103  @Override
104  public void nodeChildrenChanged(String path) {
105    if (path.equals(watcher.getZNodePaths().backupMasterAddressesZNode)) {
106      try {
107        updateBackupMasters();
108      } catch (InterruptedIOException ioe) {
109        LOG.error("Error updating backup masters", ioe);
110      }
111    }
112  }
113
114  @Override
115  public void nodeDeleted(String path) {
116    // We need to keep track of the cluster's shutdown status while
117    // we wait on the current master. We consider that, if the cluster
118    // was already in a "shutdown" state when we started, that this master
119    // is part of a new cluster that was started shortly after the old cluster
120    // shut down, so that state is now irrelevant. This means that the shutdown
121    // state must be set while we wait on the active master in order
122    // to shutdown this master. See HBASE-8519.
123    if (path.equals(watcher.getZNodePaths().clusterStateZNode) && !master.isStopped()) {
124      clusterShutDown.set(true);
125    }
126    handle(path);
127  }
128
129  void handle(final String path) {
130    if (path.equals(watcher.getZNodePaths().masterAddressZNode) && !master.isStopped()) {
131      handleMasterNodeChange();
132    }
133  }
134
135  private void updateBackupMasters() throws InterruptedIOException {
136    backupMasters =
137      ImmutableList.copyOf(MasterAddressTracker.getBackupMastersAndRenewWatch(watcher));
138  }
139
140  /**
141   * Fetches the active master's ServerName from zookeeper.
142   */
143  private void fetchAndSetActiveMasterServerName() {
144    LOG.debug("Attempting to fetch active master sn from zk");
145    try {
146      activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
147    } catch (IOException | KeeperException e) {
148      // Log and ignore for now and re-fetch later if needed.
149      LOG.error("Error fetching active master information", e);
150    }
151  }
152
153  public Optional<ServerName> getActiveMasterServerName() {
154    if (!clusterHasActiveMaster.get()) {
155      return Optional.empty();
156    }
157    if (activeMasterServerName == null) {
158      fetchAndSetActiveMasterServerName();
159    }
160    // It could still be null, but return whatever we have.
161    return Optional.ofNullable(activeMasterServerName);
162  }
163
164  public int getActiveMasterInfoPort() {
165    try {
166      return MasterAddressTracker.getMasterInfoPort(watcher);
167    } catch (Exception e) {
168      LOG.warn("Failed to get active master's info port.", e);
169      return 0;
170    }
171  }
172
173  public int getBackupMasterInfoPort(final ServerName sn) {
174    try {
175      return MasterAddressTracker.getBackupMasterInfoPort(watcher, sn);
176    } catch (Exception e) {
177      LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
178      return 0;
179    }
180  }
181
182  /**
183   * Handle a change in the master node. Doesn't matter whether this was called from a nodeCreated
184   * or nodeDeleted event because there are no guarantees that the current state of the master node
185   * matches the event at the time of our next ZK request.
186   * <p>
187   * Uses the watchAndCheckExists method which watches the master address node regardless of whether
188   * it exists or not. If it does exist (there is an active master), it returns true. Otherwise it
189   * returns false.
190   * <p>
191   * A watcher is set which guarantees that this method will get called again if there is another
192   * change in the master node.
193   */
194  private void handleMasterNodeChange() {
195    // Watch the node and check if it exists.
196    try {
197      synchronized (clusterHasActiveMaster) {
198        if (ZKUtil.watchAndCheckExists(watcher, watcher.getZNodePaths().masterAddressZNode)) {
199          // A master node exists, there is an active master
200          LOG.trace("A master is now available");
201          clusterHasActiveMaster.set(true);
202        } else {
203          // Node is no longer there, cluster does not have an active master
204          LOG.debug("No master available. Notifying waiting threads");
205          clusterHasActiveMaster.set(false);
206          // Notify any thread waiting to become the active master
207          clusterHasActiveMaster.notifyAll();
208        }
209        // Reset the active master sn. Will be re-fetched later if needed.
210        // We don't want to make a synchronous RPC under a monitor.
211        activeMasterServerName = null;
212      }
213    } catch (KeeperException ke) {
214      master.abort("Received an unexpected KeeperException, aborting", ke);
215    }
216  }
217
218  /**
219   * Block until becoming the active master. Method blocks until there is not another active master
220   * and our attempt to become the new active master is successful. This also makes sure that we are
221   * watching the master znode so will be notified if another master dies.
222   * @param checkInterval    the interval to check if the master is stopped
223   * @param startupTaskGroup the task group for master startup to track the progress
224   * @return True if no issue becoming active master else false if another master was running or if
225   *         some other problem (zookeeper, stop flag has been set on this Master)
226   */
227  boolean blockUntilBecomingActiveMaster(int checkInterval, TaskGroup startupTaskGroup) {
228    MonitoredTask blockUntilActive =
229      startupTaskGroup.addTask("Blocking until becoming active master");
230    String backupZNode = ZNodePaths
231      .joinZNode(this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString());
232    while (!(master.isAborted() || master.isStopped())) {
233      blockUntilActive.setStatus("Trying to register in ZK as active master");
234      // Try to become the active master, watch if there is another master.
235      // Write out our ServerName as versioned bytes.
236      try {
237        if (
238          MasterAddressTracker.setMasterAddress(this.watcher,
239            this.watcher.getZNodePaths().masterAddressZNode, this.sn, infoPort)
240        ) {
241
242          // If we were a backup master before, delete our ZNode from the backup
243          // master directory since we are the active now)
244          if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
245            LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
246            ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
247          }
248          // Save the znode in a file, this will allow to check if we crash in the launch scripts
249          ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
250
251          // We are the master, return
252          blockUntilActive.setStatus("Successfully registered as active master.");
253          this.clusterHasActiveMaster.set(true);
254          activeMasterServerName = sn;
255          LOG.info("Registered as active master=" + this.sn);
256          return true;
257        }
258
259        // Invalidate the active master name so that subsequent requests do not get any stale
260        // master information. Will be re-fetched if needed.
261        activeMasterServerName = null;
262        // There is another active master running elsewhere or this is a restart
263        // and the master ephemeral node has not expired yet.
264        this.clusterHasActiveMaster.set(true);
265
266        String msg;
267        byte[] bytes =
268          ZKUtil.getDataAndWatch(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);
269        if (bytes == null) {
270          msg = ("A master was detected, but went down before its address "
271            + "could be read.  Attempting to become the next active master");
272        } else {
273          ServerName currentMaster;
274          try {
275            currentMaster = ProtobufUtil.parseServerNameFrom(bytes);
276          } catch (DeserializationException e) {
277            LOG.warn("Failed parse", e);
278            // Hopefully next time around we won't fail the parse. Dangerous.
279            continue;
280          }
281          if (ServerName.isSameAddress(currentMaster, this.sn)) {
282            msg = ("Current master has this master's address, " + currentMaster
283              + "; master was restarted? Deleting node.");
284            // Hurry along the expiration of the znode.
285            ZKUtil.deleteNode(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);
286
287            // We may have failed to delete the znode at the previous step, but
288            // we delete the file anyway: a second attempt to delete the znode is likely to fail
289            // again.
290            ZNodeClearer.deleteMyEphemeralNodeOnDisk();
291          } else {
292            msg = "Another master is the active master, " + currentMaster
293              + "; waiting to become the next active master";
294          }
295        }
296        LOG.info(msg);
297        blockUntilActive.setStatus(msg);
298      } catch (KeeperException ke) {
299        master.abort("Received an unexpected KeeperException, aborting", ke);
300        return false;
301      }
302      synchronized (this.clusterHasActiveMaster) {
303        while (clusterHasActiveMaster.get() && !master.isStopped()) {
304          try {
305            clusterHasActiveMaster.wait(checkInterval);
306          } catch (InterruptedException e) {
307            // We expect to be interrupted when a master dies,
308            // will fall out if so
309            LOG.debug("Interrupted waiting for master to die", e);
310          }
311        }
312        if (clusterShutDown.get()) {
313          this.master.stop("Cluster went down before this master became active");
314        }
315      }
316    }
317    return false;
318  }
319
320  /** Returns True if cluster has an active master. */
321  boolean hasActiveMaster() {
322    try {
323      if (ZKUtil.checkExists(watcher, watcher.getZNodePaths().masterAddressZNode) >= 0) {
324        return true;
325      }
326    } catch (KeeperException ke) {
327      LOG.info("Received an unexpected KeeperException when checking " + "isActiveMaster : " + ke);
328    }
329    return false;
330  }
331
332  public void stop() {
333    try {
334      synchronized (clusterHasActiveMaster) {
335        // Master is already stopped, wake up the manager
336        // thread so that it can shutdown soon.
337        clusterHasActiveMaster.notifyAll();
338      }
339      // If our address is in ZK, delete it on our way out
340      ServerName activeMaster = null;
341      try {
342        activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
343      } catch (IOException e) {
344        LOG.warn("Failed get of master address: " + e.toString());
345      }
346      if (activeMaster != null && activeMaster.equals(this.sn)) {
347        ZKUtil.deleteNode(watcher, watcher.getZNodePaths().masterAddressZNode);
348        // We may have failed to delete the znode at the previous step, but
349        // we delete the file anyway: a second attempt to delete the znode is likely to fail again.
350        ZNodeClearer.deleteMyEphemeralNodeOnDisk();
351      }
352    } catch (KeeperException e) {
353      LOG.debug(this.watcher.prefix("Failed delete of our master address node; " + e.getMessage()));
354    }
355  }
356
357  /** Returns list of registered backup masters. */
358  public List<ServerName> getBackupMasters() {
359    return backupMasters;
360  }
361}