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