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 org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.hbase.Server;
024import org.apache.hadoop.hbase.ServerName;
025import org.apache.hadoop.hbase.monitoring.MonitoredTask;
026import org.apache.hadoop.hbase.util.Threads;
027import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
028import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
029import org.apache.zookeeper.KeeperException;
030import org.slf4j.Logger;
031import org.slf4j.LoggerFactory;
032
033/**
034 * An implementation of HMaster that always runs as a stand by and never transitions to active.
035 */
036public class AlwaysStandByHMaster extends HMaster {
037  /**
038   * An implementation of ActiveMasterManager that never transitions it's master to active state. It
039   * always remains as a stand by master. With the master registry implementation (HBASE-18095) it
040   * is expected to have at least one active / standby master always running at any point in time
041   * since they serve as the gateway for client connections.
042   *
043   * With this implementation, tests can simulate the scenario of not having an active master yet
044   * the client connections to the cluster succeed.
045   */
046  private static class AlwaysStandByMasterManager extends ActiveMasterManager {
047    private static final Logger LOG =
048        LoggerFactory.getLogger(AlwaysStandByMasterManager.class);
049
050    AlwaysStandByMasterManager(ZKWatcher watcher, ServerName sn, Server master)
051        throws InterruptedIOException {
052      super(watcher, sn, master);
053    }
054
055    /**
056     * An implementation that never transitions to an active master.
057     */
058    boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
059      while (!(master.isAborted() || master.isStopped())) {
060        startupStatus.setStatus("Forever looping to stay as a standby master.");
061        try {
062          activeMasterServerName = null;
063          try {
064            if (MasterAddressTracker.getMasterAddress(watcher) != null) {
065              clusterHasActiveMaster.set(true);
066            }
067          } catch (IOException e) {
068            // pass, we will get notified when some other active master creates the znode.
069          }
070          Threads.sleepWithoutInterrupt(1000);
071        } catch (KeeperException e) {
072          master.abort("Received an unexpected KeeperException, aborting", e);
073          return false;
074        }
075        synchronized (this.clusterHasActiveMaster) {
076          while (clusterHasActiveMaster.get() && !master.isStopped()) {
077            try {
078              clusterHasActiveMaster.wait(checkInterval);
079            } catch (InterruptedException e) {
080              // We expect to be interrupted when a master dies,
081              //  will fall out if so
082              LOG.debug("Interrupted waiting for master to die", e);
083            }
084          }
085          if (clusterShutDown.get()) {
086            this.master.stop(
087                "Cluster went down before this master became active");
088          }
089        }
090      }
091      return false;
092    }
093  }
094
095  public AlwaysStandByHMaster(Configuration conf) throws IOException {
096    super(conf);
097  }
098
099  protected ActiveMasterManager createActiveMasterManager(ZKWatcher zk, ServerName sn,
100      org.apache.hadoop.hbase.Server server) throws InterruptedIOException {
101    return new AlwaysStandByMasterManager(zk, sn, server);
102  }
103}