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 static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK;
021import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
022import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
023import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY;
024
025import com.google.protobuf.Descriptors;
026import com.google.protobuf.Service;
027import java.io.IOException;
028import java.io.InterruptedIOException;
029import java.lang.reflect.Constructor;
030import java.lang.reflect.InvocationTargetException;
031import java.net.InetAddress;
032import java.net.InetSocketAddress;
033import java.net.UnknownHostException;
034import java.util.ArrayList;
035import java.util.Arrays;
036import java.util.Collection;
037import java.util.Collections;
038import java.util.Comparator;
039import java.util.EnumSet;
040import java.util.HashMap;
041import java.util.Iterator;
042import java.util.List;
043import java.util.Map;
044import java.util.Objects;
045import java.util.Optional;
046import java.util.Set;
047import java.util.concurrent.ExecutionException;
048import java.util.concurrent.Future;
049import java.util.concurrent.TimeUnit;
050import java.util.concurrent.TimeoutException;
051import java.util.concurrent.atomic.AtomicInteger;
052import java.util.concurrent.locks.ReentrantLock;
053import java.util.regex.Pattern;
054import java.util.stream.Collectors;
055import javax.servlet.ServletException;
056import javax.servlet.http.HttpServlet;
057import javax.servlet.http.HttpServletRequest;
058import javax.servlet.http.HttpServletResponse;
059import org.apache.commons.io.IOUtils;
060import org.apache.commons.lang3.StringUtils;
061import org.apache.hadoop.conf.Configuration;
062import org.apache.hadoop.fs.FSDataOutputStream;
063import org.apache.hadoop.fs.Path;
064import org.apache.hadoop.hbase.ChoreService;
065import org.apache.hadoop.hbase.ClusterId;
066import org.apache.hadoop.hbase.ClusterMetrics;
067import org.apache.hadoop.hbase.ClusterMetrics.Option;
068import org.apache.hadoop.hbase.ClusterMetricsBuilder;
069import org.apache.hadoop.hbase.DoNotRetryIOException;
070import org.apache.hadoop.hbase.HBaseIOException;
071import org.apache.hadoop.hbase.HBaseInterfaceAudience;
072import org.apache.hadoop.hbase.HConstants;
073import org.apache.hadoop.hbase.InvalidFamilyOperationException;
074import org.apache.hadoop.hbase.MasterNotRunningException;
075import org.apache.hadoop.hbase.MetaTableAccessor;
076import org.apache.hadoop.hbase.NamespaceDescriptor;
077import org.apache.hadoop.hbase.PleaseHoldException;
078import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
079import org.apache.hadoop.hbase.ServerName;
080import org.apache.hadoop.hbase.TableName;
081import org.apache.hadoop.hbase.TableNotDisabledException;
082import org.apache.hadoop.hbase.TableNotFoundException;
083import org.apache.hadoop.hbase.UnknownRegionException;
084import org.apache.hadoop.hbase.client.Admin;
085import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
086import org.apache.hadoop.hbase.client.MasterSwitchType;
087import org.apache.hadoop.hbase.client.RegionInfo;
088import org.apache.hadoop.hbase.client.RegionInfoBuilder;
089import org.apache.hadoop.hbase.client.RegionStatesCount;
090import org.apache.hadoop.hbase.client.TableDescriptor;
091import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
092import org.apache.hadoop.hbase.client.TableState;
093import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
094import org.apache.hadoop.hbase.exceptions.DeserializationException;
095import org.apache.hadoop.hbase.executor.ExecutorType;
096import org.apache.hadoop.hbase.favored.FavoredNodesManager;
097import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
098import org.apache.hadoop.hbase.http.InfoServer;
099import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
100import org.apache.hadoop.hbase.ipc.RpcServer;
101import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
102import org.apache.hadoop.hbase.log.HBaseMarkers;
103import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
104import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
105import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
106import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
107import org.apache.hadoop.hbase.master.assignment.RegionStates;
108import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
109import org.apache.hadoop.hbase.master.balancer.BalancerChore;
110import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
111import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
112import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
113import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
114import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
115import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
116import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
117import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore;
118import org.apache.hadoop.hbase.master.locking.LockManager;
119import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
120import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
121import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
122import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
123import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
124import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
125import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
126import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
127import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
128import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
129import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
130import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
131import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
132import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
133import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
134import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
135import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
136import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
137import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
138import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
139import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
140import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
141import org.apache.hadoop.hbase.master.region.MasterRegion;
142import org.apache.hadoop.hbase.master.region.MasterRegionFactory;
143import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
144import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
145import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
146import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
147import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
148import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
149import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
150import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService;
151import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
152import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
153import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
154import org.apache.hadoop.hbase.mob.MobConstants;
155import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
156import org.apache.hadoop.hbase.monitoring.MonitoredTask;
157import org.apache.hadoop.hbase.monitoring.TaskMonitor;
158import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
159import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
160import org.apache.hadoop.hbase.procedure2.LockedResource;
161import org.apache.hadoop.hbase.procedure2.Procedure;
162import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
163import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
164import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
165import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
166import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
167import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;
168import org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore;
169import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
170import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;
171import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
172import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
173import org.apache.hadoop.hbase.quotas.QuotaUtil;
174import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
175import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
176import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
177import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
178import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
179import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
180import org.apache.hadoop.hbase.regionserver.HRegionServer;
181import org.apache.hadoop.hbase.regionserver.RSRpcServices;
182import org.apache.hadoop.hbase.replication.ReplicationException;
183import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
184import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
185import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
186import org.apache.hadoop.hbase.replication.ReplicationUtils;
187import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
188import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
189import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
190import org.apache.hadoop.hbase.replication.regionserver.ReplicationStatus;
191import org.apache.hadoop.hbase.security.AccessDeniedException;
192import org.apache.hadoop.hbase.security.SecurityConstants;
193import org.apache.hadoop.hbase.security.UserProvider;
194import org.apache.hadoop.hbase.trace.TraceUtil;
195import org.apache.hadoop.hbase.util.Addressing;
196import org.apache.hadoop.hbase.util.Bytes;
197import org.apache.hadoop.hbase.util.HBaseFsck;
198import org.apache.hadoop.hbase.util.HFileArchiveUtil;
199import org.apache.hadoop.hbase.util.IdLock;
200import org.apache.hadoop.hbase.util.ModifyRegionUtils;
201import org.apache.hadoop.hbase.util.Pair;
202import org.apache.hadoop.hbase.util.RetryCounter;
203import org.apache.hadoop.hbase.util.RetryCounterFactory;
204import org.apache.hadoop.hbase.util.TableDescriptorChecker;
205import org.apache.hadoop.hbase.util.Threads;
206import org.apache.hadoop.hbase.util.VersionInfo;
207import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
208import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
209import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
210import org.apache.hadoop.hbase.zookeeper.SnapshotCleanupTracker;
211import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
212import org.apache.hadoop.hbase.zookeeper.ZKUtil;
213import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
214import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
215import org.apache.yetus.audience.InterfaceAudience;
216import org.apache.zookeeper.KeeperException;
217import org.eclipse.jetty.server.Server;
218import org.eclipse.jetty.server.ServerConnector;
219import org.eclipse.jetty.servlet.ServletHolder;
220import org.eclipse.jetty.webapp.WebAppContext;
221import org.slf4j.Logger;
222import org.slf4j.LoggerFactory;
223
224import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
225import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
226import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
227import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
228
229import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
230import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
231import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
232
233/**
234 * HMaster is the "master server" for HBase. An HBase cluster has one active
235 * master.  If many masters are started, all compete.  Whichever wins goes on to
236 * run the cluster.  All others park themselves in their constructor until
237 * master or cluster shutdown or until the active master loses its lease in
238 * zookeeper.  Thereafter, all running master jostle to take over master role.
239 *
240 * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
241 * this case it will tell all regionservers to go down and then wait on them
242 * all reporting in that they are down.  This master will then shut itself down.
243 *
244 * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
245 *
246 * @see org.apache.zookeeper.Watcher
247 */
248@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
249@SuppressWarnings("deprecation")
250public class HMaster extends HRegionServer implements MasterServices {
251  private static Logger LOG = LoggerFactory.getLogger(HMaster.class);
252
253  /**
254   * Protection against zombie master. Started once Master accepts active responsibility and
255   * starts taking over responsibilities. Allows a finite time window before giving up ownership.
256   */
257  private static class InitializationMonitor extends Thread {
258    /** The amount of time in milliseconds to sleep before checking initialization status. */
259    public static final String TIMEOUT_KEY = "hbase.master.initializationmonitor.timeout";
260    public static final long TIMEOUT_DEFAULT = TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);
261
262    /**
263     * When timeout expired and initialization has not complete, call {@link System#exit(int)} when
264     * true, do nothing otherwise.
265     */
266    public static final String HALT_KEY = "hbase.master.initializationmonitor.haltontimeout";
267    public static final boolean HALT_DEFAULT = false;
268
269    private final HMaster master;
270    private final long timeout;
271    private final boolean haltOnTimeout;
272
273    /** Creates a Thread that monitors the {@link #isInitialized()} state. */
274    InitializationMonitor(HMaster master) {
275      super("MasterInitializationMonitor");
276      this.master = master;
277      this.timeout = master.getConfiguration().getLong(TIMEOUT_KEY, TIMEOUT_DEFAULT);
278      this.haltOnTimeout = master.getConfiguration().getBoolean(HALT_KEY, HALT_DEFAULT);
279      this.setDaemon(true);
280    }
281
282    @Override
283    public void run() {
284      try {
285        while (!master.isStopped() && master.isActiveMaster()) {
286          Thread.sleep(timeout);
287          if (master.isInitialized()) {
288            LOG.debug("Initialization completed within allotted tolerance. Monitor exiting.");
289          } else {
290            LOG.error("Master failed to complete initialization after " + timeout + "ms. Please"
291                + " consider submitting a bug report including a thread dump of this process.");
292            if (haltOnTimeout) {
293              LOG.error("Zombie Master exiting. Thread dump to stdout");
294              Threads.printThreadInfo(System.out, "Zombie HMaster");
295              System.exit(-1);
296            }
297          }
298        }
299      } catch (InterruptedException ie) {
300        LOG.trace("InitMonitor thread interrupted. Existing.");
301      }
302    }
303  }
304
305  // MASTER is name of the webapp and the attribute name used stuffing this
306  //instance into web context.
307  public static final String MASTER = "master";
308
309  // Manager and zk listener for master election
310  private final ActiveMasterManager activeMasterManager;
311  // Region server tracker
312  private RegionServerTracker regionServerTracker;
313  // Draining region server tracker
314  private DrainingServerTracker drainingServerTracker;
315  // Tracker for load balancer state
316  LoadBalancerTracker loadBalancerTracker;
317  // Tracker for meta location, if any client ZK quorum specified
318  MetaLocationSyncer metaLocationSyncer;
319  // Tracker for active master location, if any client ZK quorum specified
320  MasterAddressSyncer masterAddressSyncer;
321  // Tracker for auto snapshot cleanup state
322  SnapshotCleanupTracker snapshotCleanupTracker;
323
324  // Tracker for split and merge state
325  private SplitOrMergeTracker splitOrMergeTracker;
326
327  // Tracker for region normalizer state
328  private RegionNormalizerTracker regionNormalizerTracker;
329
330  private ClusterSchemaService clusterSchemaService;
331
332  public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS =
333    "hbase.master.wait.on.service.seconds";
334  public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS = 5 * 60;
335
336  public static final String HBASE_MASTER_CLEANER_INTERVAL = "hbase.master.cleaner.interval";
337
338  public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL = 600 * 1000;
339
340  // Metrics for the HMaster
341  final MetricsMaster metricsMaster;
342  // file system manager for the master FS operations
343  private MasterFileSystem fileSystemManager;
344  private MasterWalManager walManager;
345
346  // manager to manage procedure-based WAL splitting, can be null if current
347  // is zk-based WAL splitting. SplitWALManager will replace SplitLogManager
348  // and MasterWalManager, which means zk-based WAL splitting code will be
349  // useless after we switch to the procedure-based one. our eventual goal
350  // is to remove all the zk-based WAL splitting code.
351  private SplitWALManager splitWALManager;
352
353  // server manager to deal with region server info
354  private volatile ServerManager serverManager;
355
356  // manager of assignment nodes in zookeeper
357  private AssignmentManager assignmentManager;
358
359  /**
360   * Cache for the meta region replica's locations. Also tracks their changes to avoid stale
361   * cache entries.
362   */
363  private final MetaRegionLocationCache metaRegionLocationCache;
364
365  // manager of replication
366  private ReplicationPeerManager replicationPeerManager;
367
368  // buffer for "fatal error" notices from region servers
369  // in the cluster. This is only used for assisting
370  // operations/debugging.
371  MemoryBoundedLogMessageBuffer rsFatals;
372
373  // flag set after we become the active master (used for testing)
374  private volatile boolean activeMaster = false;
375
376  // flag set after we complete initialization once active
377  private final ProcedureEvent<?> initialized = new ProcedureEvent<>("master initialized");
378
379  // flag set after master services are started,
380  // initialization may have not completed yet.
381  volatile boolean serviceStarted = false;
382
383  // Maximum time we should run balancer for
384  private final int maxBalancingTime;
385  // Maximum percent of regions in transition when balancing
386  private final double maxRitPercent;
387
388  private final LockManager lockManager = new LockManager(this);
389
390  private LoadBalancer balancer;
391  // a lock to prevent concurrent normalization actions.
392  private final ReentrantLock normalizationInProgressLock = new ReentrantLock();
393  private RegionNormalizer normalizer;
394  private BalancerChore balancerChore;
395  private RegionNormalizerChore normalizerChore;
396  private ClusterStatusChore clusterStatusChore;
397  private ClusterStatusPublisher clusterStatusPublisherChore = null;
398  private SnapshotCleanerChore snapshotCleanerChore = null;
399
400  private HbckChore hbckChore;
401  CatalogJanitor catalogJanitorChore;
402  private DirScanPool cleanerPool;
403  private LogCleaner logCleaner;
404  private HFileCleaner hfileCleaner;
405  private ReplicationBarrierCleaner replicationBarrierCleaner;
406  private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
407  private MobCompactionChore mobCompactChore;
408  private MasterMobCompactionThread mobCompactThread;
409  // used to synchronize the mobCompactionStates
410  private final IdLock mobCompactionLock = new IdLock();
411  // save the information of mob compactions in tables.
412  // the key is table name, the value is the number of compactions in that table.
413  private Map<TableName, AtomicInteger> mobCompactionStates = Maps.newConcurrentMap();
414
415  MasterCoprocessorHost cpHost;
416
417  private final boolean preLoadTableDescriptors;
418
419  // Time stamps for when a hmaster became active
420  private long masterActiveTime;
421
422  // Time stamp for when HMaster finishes becoming Active Master
423  private long masterFinishedInitializationTime;
424
425  Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
426
427  // monitor for snapshot of hbase tables
428  SnapshotManager snapshotManager;
429  // monitor for distributed procedures
430  private MasterProcedureManagerHost mpmHost;
431
432  private RegionsRecoveryChore regionsRecoveryChore = null;
433
434  private RegionsRecoveryConfigManager regionsRecoveryConfigManager = null;
435  // it is assigned after 'initialized' guard set to true, so should be volatile
436  private volatile MasterQuotaManager quotaManager;
437  private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
438  private QuotaObserverChore quotaObserverChore;
439  private SnapshotQuotaObserverChore snapshotQuotaChore;
440
441  private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
442  private ProcedureStore procedureStore;
443
444  // the master local storage to store procedure data, etc.
445  private MasterRegion masterRegion;
446
447  // handle table states
448  private TableStateManager tableStateManager;
449
450  private long splitPlanCount;
451  private long mergePlanCount;
452
453  /* Handle favored nodes information */
454  private FavoredNodesManager favoredNodesManager;
455
456  /** jetty server for master to redirect requests to regionserver infoServer */
457  private Server masterJettyServer;
458
459  // Determine if we should do normal startup or minimal "single-user" mode with no region
460  // servers and no user tables. Useful for repair and recovery of hbase:meta
461  private final boolean maintenanceMode;
462  static final String MAINTENANCE_MODE = "hbase.master.maintenance_mode";
463
464  // Cached clusterId on stand by masters to serve clusterID requests from clients.
465  private final CachedClusterId cachedClusterId;
466
467  public static class RedirectServlet extends HttpServlet {
468    private static final long serialVersionUID = 2894774810058302473L;
469    private final int regionServerInfoPort;
470    private final String regionServerHostname;
471
472    /**
473     * @param infoServer that we're trying to send all requests to
474     * @param hostname may be null. if given, will be used for redirects instead of host from client.
475     */
476    public RedirectServlet(InfoServer infoServer, String hostname) {
477       regionServerInfoPort = infoServer.getPort();
478       regionServerHostname = hostname;
479    }
480
481    @Override
482    public void doGet(HttpServletRequest request,
483        HttpServletResponse response) throws ServletException, IOException {
484      String redirectHost = regionServerHostname;
485      if(redirectHost == null) {
486        redirectHost = request.getServerName();
487        if(!Addressing.isLocalAddress(InetAddress.getByName(redirectHost))) {
488          LOG.warn("Couldn't resolve '" + redirectHost + "' as an address local to this node and '" +
489              MASTER_HOSTNAME_KEY + "' is not set; client will get an HTTP 400 response. If " +
490              "your HBase deployment relies on client accessible names that the region server process " +
491              "can't resolve locally, then you should set the previously mentioned configuration variable " +
492              "to an appropriate hostname.");
493          // no sending client provided input back to the client, so the goal host is just in the logs.
494          response.sendError(400, "Request was to a host that I can't resolve for any of the network interfaces on " +
495              "this node. If this is due to an intermediary such as an HTTP load balancer or other proxy, your HBase " +
496              "administrator can set '" + MASTER_HOSTNAME_KEY + "' to point to the correct hostname.");
497          return;
498        }
499      }
500      // TODO this scheme should come from looking at the scheme registered in the infoserver's http server for the
501      // host and port we're using, but it's buried way too deep to do that ATM.
502      String redirectUrl = request.getScheme() + "://"
503        + redirectHost + ":" + regionServerInfoPort
504        + request.getRequestURI();
505      response.sendRedirect(redirectUrl);
506    }
507  }
508
509  /**
510   * Initializes the HMaster. The steps are as follows:
511   * <p>
512   * <ol>
513   * <li>Initialize the local HRegionServer
514   * <li>Start the ActiveMasterManager.
515   * </ol>
516   * <p>
517   * Remaining steps of initialization occur in
518   * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the
519   * active one.
520   */
521  public HMaster(final Configuration conf) throws IOException {
522    super(conf);
523    TraceUtil.initTracer(conf);
524    try {
525      if (conf.getBoolean(MAINTENANCE_MODE, false)) {
526        LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);
527        maintenanceMode = true;
528      } else if (Boolean.getBoolean(MAINTENANCE_MODE)) {
529        LOG.info("Detected {}=true via environment variables.", MAINTENANCE_MODE);
530        maintenanceMode = true;
531      } else {
532        maintenanceMode = false;
533      }
534      this.rsFatals = new MemoryBoundedLogMessageBuffer(
535          conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
536      LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}", getDataRootDir(),
537          this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
538
539      // Disable usage of meta replicas in the master
540      this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
541
542      decorateMasterConfiguration(this.conf);
543
544      // Hack! Maps DFSClient => Master for logs.  HDFS made this
545      // config param for task trackers, but we can piggyback off of it.
546      if (this.conf.get("mapreduce.task.attempt.id") == null) {
547        this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());
548      }
549
550      this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this));
551
552      // preload table descriptor at startup
553      this.preLoadTableDescriptors = conf.getBoolean("hbase.master.preload.tabledescriptors", true);
554
555      this.maxBalancingTime = getMaxBalancingTime();
556      this.maxRitPercent = conf.getDouble(HConstants.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT,
557          HConstants.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT);
558
559      // Do we publish the status?
560
561      boolean shouldPublish = conf.getBoolean(HConstants.STATUS_PUBLISHED,
562          HConstants.STATUS_PUBLISHED_DEFAULT);
563      Class<? extends ClusterStatusPublisher.Publisher> publisherClass =
564          conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,
565              ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,
566              ClusterStatusPublisher.Publisher.class);
567
568      if (shouldPublish) {
569        if (publisherClass == null) {
570          LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
571              ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS +
572              " is not set - not publishing status");
573        } else {
574          clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
575          LOG.debug("Created {}", this.clusterStatusPublisherChore);
576          getChoreService().scheduleChore(clusterStatusPublisherChore);
577        }
578      }
579
580      // Some unit tests don't need a cluster, so no zookeeper at all
581      if (!conf.getBoolean("hbase.testing.nocluster", false)) {
582        this.metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper);
583        this.activeMasterManager = createActiveMasterManager(zooKeeper, serverName, this);
584      } else {
585        this.metaRegionLocationCache = null;
586        this.activeMasterManager = null;
587      }
588      cachedClusterId = new CachedClusterId(this, conf);
589    } catch (Throwable t) {
590      // Make sure we log the exception. HMaster is often started via reflection and the
591      // cause of failed startup is lost.
592      LOG.error("Failed construction of Master", t);
593      throw t;
594    }
595  }
596
597  /**
598   * Protected to have custom implementations in tests override the default ActiveMaster
599   * implementation.
600   */
601  protected ActiveMasterManager createActiveMasterManager(
602      ZKWatcher zk, ServerName sn, org.apache.hadoop.hbase.Server server) {
603    return new ActiveMasterManager(zk, sn, server);
604  }
605
606  @Override
607  protected String getUseThisHostnameInstead(Configuration conf) {
608    return conf.get(MASTER_HOSTNAME_KEY);
609  }
610
611  // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
612  // block in here until then.
613  @Override
614  public void run() {
615    try {
616      if (!conf.getBoolean("hbase.testing.nocluster", false)) {
617        Threads.setDaemonThreadRunning(new Thread(() -> {
618          try {
619            int infoPort = putUpJettyServer();
620            startActiveMasterManager(infoPort);
621          } catch (Throwable t) {
622            // Make sure we log the exception.
623            String error = "Failed to become Active Master";
624            LOG.error(error, t);
625            // Abort should have been called already.
626            if (!isAborted()) {
627              abort(error, t);
628            }
629          }
630        }), getName() + ":becomeActiveMaster");
631      }
632      // Fall in here even if we have been aborted. Need to run the shutdown services and
633      // the super run call will do this for us.
634      super.run();
635    } finally {
636      if (this.clusterSchemaService != null) {
637        // If on way out, then we are no longer active master.
638        this.clusterSchemaService.stopAsync();
639        try {
640          this.clusterSchemaService.awaitTerminated(
641              getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
642              DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
643        } catch (TimeoutException te) {
644          LOG.warn("Failed shutdown of clusterSchemaService", te);
645        }
646      }
647      this.activeMaster = false;
648    }
649  }
650
651  // return the actual infoPort, -1 means disable info server.
652  private int putUpJettyServer() throws IOException {
653    if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {
654      return -1;
655    }
656    final int infoPort = conf.getInt("hbase.master.info.port.orig",
657      HConstants.DEFAULT_MASTER_INFOPORT);
658    // -1 is for disabling info server, so no redirecting
659    if (infoPort < 0 || infoServer == null) {
660      return -1;
661    }
662    if(infoPort == infoServer.getPort()) {
663      return infoPort;
664    }
665    final String addr = conf.get("hbase.master.info.bindAddress", "0.0.0.0");
666    if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
667      String msg =
668          "Failed to start redirecting jetty server. Address " + addr
669              + " does not belong to this host. Correct configuration parameter: "
670              + "hbase.master.info.bindAddress";
671      LOG.error(msg);
672      throw new IOException(msg);
673    }
674
675    // TODO I'm pretty sure we could just add another binding to the InfoServer run by
676    // the RegionServer and have it run the RedirectServlet instead of standing up
677    // a second entire stack here.
678    masterJettyServer = new Server();
679    final ServerConnector connector = new ServerConnector(masterJettyServer);
680    connector.setHost(addr);
681    connector.setPort(infoPort);
682    masterJettyServer.addConnector(connector);
683    masterJettyServer.setStopAtShutdown(true);
684
685    final String redirectHostname =
686        StringUtils.isBlank(useThisHostnameInstead) ? null : useThisHostnameInstead;
687
688    final RedirectServlet redirect = new RedirectServlet(infoServer, redirectHostname);
689    final WebAppContext context = new WebAppContext(null, "/", null, null, null, null, WebAppContext.NO_SESSIONS);
690    context.addServlet(new ServletHolder(redirect), "/*");
691    context.setServer(masterJettyServer);
692
693    try {
694      masterJettyServer.start();
695    } catch (Exception e) {
696      throw new IOException("Failed to start redirecting jetty server", e);
697    }
698    return connector.getLocalPort();
699  }
700
701  /**
702   * For compatibility, if failed with regionserver credentials, try the master one
703   */
704  @Override
705  protected void login(UserProvider user, String host) throws IOException {
706    try {
707      super.login(user, host);
708    } catch (IOException ie) {
709      user.login(SecurityConstants.MASTER_KRB_KEYTAB_FILE,
710              SecurityConstants.MASTER_KRB_PRINCIPAL, host);
711    }
712  }
713
714  /**
715   * If configured to put regions on active master,
716   * wait till a backup master becomes active.
717   * Otherwise, loop till the server is stopped or aborted.
718   */
719  @Override
720  protected void waitForMasterActive(){
721    if (maintenanceMode) {
722      return;
723    }
724    boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(conf);
725    while (!(tablesOnMaster && activeMaster) && !isStopped() && !isAborted()) {
726      sleeper.sleep();
727    }
728  }
729
730  @VisibleForTesting
731  public MasterRpcServices getMasterRpcServices() {
732    return (MasterRpcServices)rpcServices;
733  }
734
735  public boolean balanceSwitch(final boolean b) throws IOException {
736    return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC);
737  }
738
739  @Override
740  protected String getProcessName() {
741    return MASTER;
742  }
743
744  @Override
745  protected boolean canCreateBaseZNode() {
746    return true;
747  }
748
749  @Override
750  protected boolean canUpdateTableDescriptor() {
751    return true;
752  }
753
754  @Override
755  protected RSRpcServices createRpcServices() throws IOException {
756    return new MasterRpcServices(this);
757  }
758
759  @Override
760  protected void configureInfoServer() {
761    infoServer.addUnprivilegedServlet("master-status", "/master-status", MasterStatusServlet.class);
762    infoServer.setAttribute(MASTER, this);
763    if (LoadBalancer.isTablesOnMaster(conf)) {
764      super.configureInfoServer();
765    }
766  }
767
768  @Override
769  protected Class<? extends HttpServlet> getDumpServlet() {
770    return MasterDumpServlet.class;
771  }
772
773  @Override
774  public MetricsMaster getMasterMetrics() {
775    return metricsMaster;
776  }
777
778  /**
779   * <p>
780   * Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
781   * should have already been initialized along with {@link ServerManager}.
782   * </p>
783   * <p>
784   * Will be overridden in tests.
785   * </p>
786   */
787  @VisibleForTesting
788  protected void initializeZKBasedSystemTrackers()
789      throws IOException, InterruptedException, KeeperException, ReplicationException {
790    this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
791    this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
792    this.normalizer.setMasterServices(this);
793    this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
794    this.loadBalancerTracker.start();
795
796    this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
797    this.normalizer.setMasterServices(this);
798    this.regionNormalizerTracker = new RegionNormalizerTracker(zooKeeper, this);
799    this.regionNormalizerTracker.start();
800
801    this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
802    this.splitOrMergeTracker.start();
803
804    this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf);
805
806    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
807    this.drainingServerTracker.start();
808
809    this.snapshotCleanupTracker = new SnapshotCleanupTracker(zooKeeper, this);
810    this.snapshotCleanupTracker.start();
811
812    String clientQuorumServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);
813    boolean clientZkObserverMode = conf.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE,
814      HConstants.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE);
815    if (clientQuorumServers != null && !clientZkObserverMode) {
816      // we need to take care of the ZK information synchronization
817      // if given client ZK are not observer nodes
818      ZKWatcher clientZkWatcher = new ZKWatcher(conf,
819          getProcessName() + ":" + rpcServices.getSocketAddress().getPort() + "-clientZK", this,
820          false, true);
821      this.metaLocationSyncer = new MetaLocationSyncer(zooKeeper, clientZkWatcher, this);
822      this.metaLocationSyncer.start();
823      this.masterAddressSyncer = new MasterAddressSyncer(zooKeeper, clientZkWatcher, this);
824      this.masterAddressSyncer.start();
825      // set cluster id is a one-go effort
826      ZKClusterId.setClusterId(clientZkWatcher, fileSystemManager.getClusterId());
827    }
828
829    // Set the cluster as up.  If new RSs, they'll be waiting on this before
830    // going ahead with their startup.
831    boolean wasUp = this.clusterStatusTracker.isClusterUp();
832    if (!wasUp) this.clusterStatusTracker.setClusterUp();
833
834    LOG.info("Active/primary master=" + this.serverName +
835        ", sessionid=0x" +
836        Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
837        ", setting cluster-up flag (Was=" + wasUp + ")");
838
839    // create/initialize the snapshot manager and other procedure managers
840    this.snapshotManager = new SnapshotManager();
841    this.mpmHost = new MasterProcedureManagerHost();
842    this.mpmHost.register(this.snapshotManager);
843    this.mpmHost.register(new MasterFlushTableProcedureManager());
844    this.mpmHost.loadProcedures(conf);
845    this.mpmHost.initialize(this, this.metricsMaster);
846  }
847
848  // Will be overriden in test to inject customized AssignmentManager
849  @VisibleForTesting
850  protected AssignmentManager createAssignmentManager(MasterServices master) {
851    return new AssignmentManager(master);
852  }
853
854  /**
855   * Finish initialization of HMaster after becoming the primary master.
856   * <p/>
857   * The startup order is a bit complicated but very important, do not change it unless you know
858   * what you are doing.
859   * <ol>
860   * <li>Initialize file system based components - file system manager, wal manager, table
861   * descriptors, etc</li>
862   * <li>Publish cluster id</li>
863   * <li>Here comes the most complicated part - initialize server manager, assignment manager and
864   * region server tracker
865   * <ol type='i'>
866   * <li>Create server manager</li>
867   * <li>Create procedure executor, load the procedures, but do not start workers. We will start it
868   * later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
869   * server</li>
870   * <li>Create assignment manager and start it, load the meta region state, but do not load data
871   * from meta region</li>
872   * <li>Start region server tracker, construct the online servers set and find out dead servers and
873   * schedule SCP for them. The online servers will be constructed by scanning zk, and we will also
874   * scan the wal directory to find out possible live region servers, and the differences between
875   * these two sets are the dead servers</li>
876   * </ol>
877   * </li>
878   * <li>If this is a new deploy, schedule a InitMetaProcedure to initialize meta</li>
879   * <li>Start necessary service threads - balancer, catalog janior, executor services, and also the
880   * procedure executor, etc. Notice that the balancer must be created first as assignment manager
881   * may use it when assigning regions.</li>
882   * <li>Wait for meta to be initialized if necesssary, start table state manager.</li>
883   * <li>Wait for enough region servers to check-in</li>
884   * <li>Let assignment manager load data from meta and construct region states</li>
885   * <li>Start all other things such as chore services, etc</li>
886   * </ol>
887   * <p/>
888   * Notice that now we will not schedule a special procedure to make meta online(unless the first
889   * time where meta has not been created yet), we will rely on SCP to bring meta online.
890   */
891  private void finishActiveMasterInitialization(MonitoredTask status)
892      throws IOException, InterruptedException, KeeperException, ReplicationException {
893    /*
894     * We are active master now... go initialize components we need to run.
895     */
896    status.setStatus("Initializing Master file system");
897
898    this.masterActiveTime = System.currentTimeMillis();
899    // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
900
901    // always initialize the MemStoreLAB as we use a region to store data in master now, see
902    // localStore.
903    initializeMemStoreChunkCreator();
904    this.fileSystemManager = new MasterFileSystem(conf);
905    this.walManager = new MasterWalManager(this);
906
907    // enable table descriptors cache
908    this.tableDescriptors.setCacheOn();
909
910    // warm-up HTDs cache on master initialization
911    if (preLoadTableDescriptors) {
912      status.setStatus("Pre-loading table descriptors");
913      this.tableDescriptors.getAll();
914    }
915
916    // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
917    // only after it has checked in with the Master. At least a few tests ask Master for clusterId
918    // before it has called its run method and before RegionServer has done the reportForDuty.
919    ClusterId clusterId = fileSystemManager.getClusterId();
920    status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
921    ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
922    this.clusterId = clusterId.toString();
923
924    // Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their
925    // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
926    // hbase.write.hbck1.lock.file to false.
927    if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) {
928      Pair<Path, FSDataOutputStream> result = null;
929      try {
930        result = HBaseFsck.checkAndMarkRunningHbck(this.conf,
931            HBaseFsck.createLockRetryCounterFactory(this.conf).create());
932      } finally {
933        if (result != null) {
934          IOUtils.closeQuietly(result.getSecond());
935        }
936      }
937    }
938
939    status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
940    this.serverManager = createServerManager(this);
941    if (!conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK,
942      DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)) {
943      this.splitWALManager = new SplitWALManager(this);
944    }
945
946    // initialize master local region
947    masterRegion = MasterRegionFactory.create(this);
948    createProcedureExecutor();
949    Map<Class<?>, List<Procedure<MasterProcedureEnv>>> procsByType =
950      procedureExecutor.getActiveProceduresNoCopy().stream()
951        .collect(Collectors.groupingBy(p -> p.getClass()));
952
953    // Create Assignment Manager
954    this.assignmentManager = createAssignmentManager(this);
955    this.assignmentManager.start();
956    // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
957    // completed, it could still be in the procedure list. This is a bit strange but is another
958    // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.
959    List<TransitRegionStateProcedure> ritList =
960      procsByType.getOrDefault(TransitRegionStateProcedure.class, Collections.emptyList()).stream()
961        .filter(p -> !p.isFinished()).map(p -> (TransitRegionStateProcedure) p)
962        .collect(Collectors.toList());
963    this.assignmentManager.setupRIT(ritList);
964
965    // Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
966    // be registered in the deadServers set -- and with the list of servernames out on the
967    // filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
968    // We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
969    // TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
970    this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
971    this.regionServerTracker.start(
972      procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
973        .map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()),
974      walManager.getLiveServersFromWALDir(), walManager.getSplittingServersFromWALDir());
975    // This manager will be started AFTER hbase:meta is confirmed on line.
976    // hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table
977    // state from zookeeper while hbase2 reads it from hbase:meta. Disable if no hbase1 clients.
978    this.tableStateManager =
979      this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, true)
980        ?
981        new MirroringTableStateManager(this):
982        new TableStateManager(this);
983
984    status.setStatus("Initializing ZK system trackers");
985    initializeZKBasedSystemTrackers();
986    // Set ourselves as active Master now our claim has succeeded up in zk.
987    this.activeMaster = true;
988
989    // Start the Zombie master detector after setting master as active, see HBASE-21535
990    Thread zombieDetector = new Thread(new InitializationMonitor(this),
991        "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
992    zombieDetector.setDaemon(true);
993    zombieDetector.start();
994
995    // This is for backwards compatibility
996    // See HBASE-11393
997    status.setStatus("Update TableCFs node in ZNode");
998    ReplicationPeerConfigUpgrader tableCFsUpdater =
999        new ReplicationPeerConfigUpgrader(zooKeeper, conf);
1000    tableCFsUpdater.copyTableCFs();
1001
1002    if (!maintenanceMode) {
1003      // Add the Observer to delete quotas on table deletion before starting all CPs by
1004      // default with quota support, avoiding if user specifically asks to not load this Observer.
1005      if (QuotaUtil.isQuotaEnabled(conf)) {
1006        updateConfigurationForQuotasObserver(conf);
1007      }
1008      // initialize master side coprocessors before we start handling requests
1009      status.setStatus("Initializing master coprocessors");
1010      this.cpHost = new MasterCoprocessorHost(this, this.conf);
1011    }
1012
1013    // Checking if meta needs initializing.
1014    status.setStatus("Initializing meta table if this is a new deploy");
1015    InitMetaProcedure initMetaProc = null;
1016    // Print out state of hbase:meta on startup; helps debugging.
1017    RegionState rs = this.assignmentManager.getRegionStates().
1018        getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
1019    LOG.info("hbase:meta {}", rs);
1020    if (rs != null && rs.isOffline()) {
1021      Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
1022        .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
1023      initMetaProc = optProc.orElseGet(() -> {
1024        // schedule an init meta procedure if meta has not been deployed yet
1025        InitMetaProcedure temp = new InitMetaProcedure();
1026        procedureExecutor.submitProcedure(temp);
1027        return temp;
1028      });
1029    }
1030    if (this.balancer instanceof FavoredNodesPromoter) {
1031      favoredNodesManager = new FavoredNodesManager(this);
1032    }
1033
1034    // initialize load balancer
1035    this.balancer.setMasterServices(this);
1036    this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
1037    this.balancer.initialize();
1038
1039    // start up all service threads.
1040    status.setStatus("Initializing master service threads");
1041    startServiceThreads();
1042    // wait meta to be initialized after we start procedure executor
1043    if (initMetaProc != null) {
1044      initMetaProc.await();
1045    }
1046    // Wake up this server to check in
1047    sleeper.skipSleepCycle();
1048
1049    // Wait for region servers to report in.
1050    // With this as part of master initialization, it precludes our being able to start a single
1051    // server that is both Master and RegionServer. Needs more thought. TODO.
1052    String statusStr = "Wait for region servers to report in";
1053    status.setStatus(statusStr);
1054    LOG.info(Objects.toString(status));
1055    waitForRegionServers(status);
1056
1057    // Check if master is shutting down because issue initializing regionservers or balancer.
1058    if (isStopped()) {
1059      return;
1060    }
1061
1062    status.setStatus("Starting assignment manager");
1063    // FIRST HBASE:META READ!!!!
1064    // The below cannot make progress w/o hbase:meta being online.
1065    // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
1066    // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
1067    // if it is down. It may take a while to come online. So, wait here until meta if for sure
1068    // available. That's what waitForMetaOnline does.
1069    if (!waitForMetaOnline()) {
1070      return;
1071    }
1072    this.assignmentManager.joinCluster();
1073    // The below depends on hbase:meta being online.
1074    this.tableStateManager.start();
1075    // Below has to happen after tablestatemanager has started in the case where this hbase-2.x
1076    // is being started over an hbase-1.x dataset. tablestatemanager runs a migration as part
1077    // of its 'start' moving table state from zookeeper to hbase:meta. This migration needs to
1078    // complete before we do this next step processing offline regions else it fails reading
1079    // table states messing up master launch (namespace table, etc., are not assigned).
1080    this.assignmentManager.processOfflineRegions();
1081    // Initialize after meta is up as below scans meta
1082    if (favoredNodesManager != null && !maintenanceMode) {
1083      SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
1084          new SnapshotOfRegionAssignmentFromMeta(getConnection());
1085      snapshotOfRegionAssignment.initialize();
1086      favoredNodesManager.initialize(snapshotOfRegionAssignment);
1087    }
1088
1089    // set cluster status again after user regions are assigned
1090    this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
1091
1092    // Start balancer and meta catalog janitor after meta and regions have been assigned.
1093    status.setStatus("Starting balancer and catalog janitor");
1094    this.clusterStatusChore = new ClusterStatusChore(this, balancer);
1095    getChoreService().scheduleChore(clusterStatusChore);
1096    this.balancerChore = new BalancerChore(this);
1097    getChoreService().scheduleChore(balancerChore);
1098    this.normalizerChore = new RegionNormalizerChore(this);
1099    getChoreService().scheduleChore(normalizerChore);
1100    this.catalogJanitorChore = new CatalogJanitor(this);
1101    getChoreService().scheduleChore(catalogJanitorChore);
1102    this.hbckChore = new HbckChore(this);
1103    getChoreService().scheduleChore(hbckChore);
1104
1105    // NAMESPACE READ!!!!
1106    // Here we expect hbase:namespace to be online. See inside initClusterSchemaService.
1107    // TODO: Fix this. Namespace is a pain being a sort-of system table. Fold it in to hbase:meta.
1108    // isNamespace does like isMeta and waits until namespace is onlined before allowing progress.
1109    if (!waitForNamespaceOnline()) {
1110      return;
1111    }
1112    status.setStatus("Starting cluster schema service");
1113    initClusterSchemaService();
1114
1115    if (this.cpHost != null) {
1116      try {
1117        this.cpHost.preMasterInitialization();
1118      } catch (IOException e) {
1119        LOG.error("Coprocessor preMasterInitialization() hook failed", e);
1120      }
1121    }
1122
1123    status.markComplete("Initialization successful");
1124    LOG.info(String.format("Master has completed initialization %.3fsec",
1125       (System.currentTimeMillis() - masterActiveTime) / 1000.0f));
1126    this.masterFinishedInitializationTime = System.currentTimeMillis();
1127    configurationManager.registerObserver(this.balancer);
1128    configurationManager.registerObserver(this.cleanerPool);
1129    configurationManager.registerObserver(this.hfileCleaner);
1130    configurationManager.registerObserver(this.logCleaner);
1131    configurationManager.registerObserver(this.regionsRecoveryConfigManager);
1132    // Set master as 'initialized'.
1133    setInitialized(true);
1134
1135    if (maintenanceMode) {
1136      LOG.info("Detected repair mode, skipping final initialization steps.");
1137      return;
1138    }
1139
1140    assignmentManager.checkIfShouldMoveSystemRegionAsync();
1141    status.setStatus("Assign meta replicas");
1142    MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
1143    try {
1144      metaBootstrap.assignMetaReplicas();
1145    } catch (IOException | KeeperException e){
1146      LOG.error("Assigning meta replica failed: ", e);
1147    }
1148    status.setStatus("Starting quota manager");
1149    initQuotaManager();
1150    if (QuotaUtil.isQuotaEnabled(conf)) {
1151      // Create the quota snapshot notifier
1152      spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
1153      spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
1154      this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
1155      // Start the chore to read the region FS space reports and act on them
1156      getChoreService().scheduleChore(quotaObserverChore);
1157
1158      this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());
1159      // Start the chore to read snapshots and add their usage to table/NS quotas
1160      getChoreService().scheduleChore(snapshotQuotaChore);
1161    }
1162    final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);
1163    slowLogMasterService.init();
1164
1165    // clear the dead servers with same host name and port of online server because we are not
1166    // removing dead server with same hostname and port of rs which is trying to check in before
1167    // master initialization. See HBASE-5916.
1168    this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
1169
1170    // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1171    status.setStatus("Checking ZNode ACLs");
1172    zooKeeper.checkAndSetZNodeAcls();
1173
1174    status.setStatus("Initializing MOB Cleaner");
1175    initMobCleaner();
1176
1177    status.setStatus("Calling postStartMaster coprocessors");
1178    if (this.cpHost != null) {
1179      // don't let cp initialization errors kill the master
1180      try {
1181        this.cpHost.postStartMaster();
1182      } catch (IOException ioe) {
1183        LOG.error("Coprocessor postStartMaster() hook failed", ioe);
1184      }
1185    }
1186
1187    zombieDetector.interrupt();
1188
1189    /*
1190     * After master has started up, lets do balancer post startup initialization. Since this runs
1191     * in activeMasterManager thread, it should be fine.
1192     */
1193    long start = System.currentTimeMillis();
1194    this.balancer.postMasterStartupInitialize();
1195    if (LOG.isDebugEnabled()) {
1196      LOG.debug("Balancer post startup initialization complete, took " + (
1197          (System.currentTimeMillis() - start) / 1000) + " seconds");
1198    }
1199  }
1200
1201  /**
1202   * Check hbase:meta is up and ready for reading. For use during Master startup only.
1203   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
1204   *   and we will hold here until operator intervention.
1205   */
1206  @VisibleForTesting
1207  public boolean waitForMetaOnline() throws InterruptedException {
1208    return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
1209  }
1210
1211  /**
1212   * @return True if region is online and scannable else false if an error or shutdown (Otherwise
1213   *   we just block in here holding up all forward-progess).
1214   */
1215  private boolean isRegionOnline(RegionInfo ri) throws InterruptedException {
1216    RetryCounter rc = null;
1217    while (!isStopped()) {
1218      RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);
1219      if (rs.isOpened()) {
1220        if (this.getServerManager().isServerOnline(rs.getServerName())) {
1221          return true;
1222        }
1223      }
1224      // Region is not OPEN.
1225      Optional<Procedure<MasterProcedureEnv>> optProc = this.procedureExecutor.getProcedures().
1226          stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
1227      // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
1228      // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
1229      // then how to assign including how to break region lock if one held.
1230      LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
1231          "progress, in holding-pattern until region onlined.",
1232          ri.getRegionNameAsString(), rs, optProc.isPresent());
1233      // Check once-a-minute.
1234      if (rc == null) {
1235        rc = new RetryCounterFactory(1000).create();
1236      }
1237      Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
1238    }
1239    return false;
1240  }
1241
1242  /**
1243   * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
1244   * (TODO: Fix this! NS should not hold-up startup).
1245   * @return True if namespace table is up/online.
1246   */
1247  @VisibleForTesting
1248  public boolean waitForNamespaceOnline() throws InterruptedException {
1249    List<RegionInfo> ris = this.assignmentManager.getRegionStates().
1250        getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
1251    if (ris.isEmpty()) {
1252      // If empty, means we've not assigned the namespace table yet... Just return true so startup
1253      // continues and the namespace table gets created.
1254      return true;
1255    }
1256    // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
1257    for (RegionInfo ri: ris) {
1258      isRegionOnline(ri);
1259    }
1260    return true;
1261  }
1262
1263  /**
1264   * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
1265   * automatically remove quotas for a table when that table is deleted.
1266   */
1267  @VisibleForTesting
1268  public void updateConfigurationForQuotasObserver(Configuration conf) {
1269    // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
1270    if (!conf.getBoolean(
1271          MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
1272          MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)) {
1273      return;
1274    }
1275    String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
1276    final int length = null == masterCoprocs ? 0 : masterCoprocs.length;
1277    String[] updatedCoprocs = new String[length + 1];
1278    if (length > 0) {
1279      System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);
1280    }
1281    updatedCoprocs[length] = MasterQuotasObserver.class.getName();
1282    conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);
1283  }
1284
1285  private void initMobCleaner() {
1286    this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
1287    getChoreService().scheduleChore(expiredMobFileCleanerChore);
1288
1289    int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
1290        MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
1291    this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
1292    getChoreService().scheduleChore(mobCompactChore);
1293    this.mobCompactThread = new MasterMobCompactionThread(this);
1294  }
1295
1296  /**
1297   * <p>
1298   * Create a {@link MasterMetaBootstrap} instance.
1299   * </p>
1300   * <p>
1301   * Will be overridden in tests.
1302   * </p>
1303   */
1304  @VisibleForTesting
1305  protected MasterMetaBootstrap createMetaBootstrap() {
1306    // We put this out here in a method so can do a Mockito.spy and stub it out
1307    // w/ a mocked up MasterMetaBootstrap.
1308    return new MasterMetaBootstrap(this);
1309  }
1310
1311  /**
1312   * <p>
1313   * Create a {@link ServerManager} instance.
1314   * </p>
1315   * <p>
1316   * Will be overridden in tests.
1317   * </p>
1318   */
1319  @VisibleForTesting
1320  protected ServerManager createServerManager(final MasterServices master) throws IOException {
1321    // We put this out here in a method so can do a Mockito.spy and stub it out
1322    // w/ a mocked up ServerManager.
1323    setupClusterConnection();
1324    return new ServerManager(master);
1325  }
1326
1327  private void waitForRegionServers(final MonitoredTask status)
1328      throws IOException, InterruptedException {
1329    this.serverManager.waitForRegionServers(status);
1330  }
1331
1332  // Will be overridden in tests
1333  @VisibleForTesting
1334  protected void initClusterSchemaService() throws IOException, InterruptedException {
1335    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
1336    this.clusterSchemaService.startAsync();
1337    try {
1338      this.clusterSchemaService.awaitRunning(getConfiguration().getInt(
1339        HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
1340        DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
1341    } catch (TimeoutException toe) {
1342      throw new IOException("Timedout starting ClusterSchemaService", toe);
1343    }
1344  }
1345
1346  private void initQuotaManager() throws IOException {
1347    MasterQuotaManager quotaManager = new MasterQuotaManager(this);
1348    quotaManager.start();
1349    this.quotaManager = quotaManager;
1350  }
1351
1352  private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
1353    SpaceQuotaSnapshotNotifier notifier =
1354        SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
1355    return notifier;
1356  }
1357
1358  boolean isCatalogJanitorEnabled() {
1359    return catalogJanitorChore != null ?
1360      catalogJanitorChore.getEnabled() : false;
1361  }
1362
1363  boolean isCleanerChoreEnabled() {
1364    boolean hfileCleanerFlag = true, logCleanerFlag = true;
1365
1366    if (hfileCleaner != null) {
1367      hfileCleanerFlag = hfileCleaner.getEnabled();
1368    }
1369
1370    if (logCleaner != null) {
1371      logCleanerFlag = logCleaner.getEnabled();
1372    }
1373
1374    return (hfileCleanerFlag && logCleanerFlag);
1375  }
1376
1377  @Override
1378  public ServerManager getServerManager() {
1379    return this.serverManager;
1380  }
1381
1382  @Override
1383  public MasterFileSystem getMasterFileSystem() {
1384    return this.fileSystemManager;
1385  }
1386
1387  @Override
1388  public MasterWalManager getMasterWalManager() {
1389    return this.walManager;
1390  }
1391
1392  @Override
1393  public SplitWALManager getSplitWALManager() {
1394    return splitWALManager;
1395  }
1396
1397  @Override
1398  public TableStateManager getTableStateManager() {
1399    return tableStateManager;
1400  }
1401
1402  /*
1403   * Start up all services. If any of these threads gets an unhandled exception
1404   * then they just die with a logged message.  This should be fine because
1405   * in general, we do not expect the master to get such unhandled exceptions
1406   *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
1407   *  need to install an unexpected exception handler.
1408   */
1409  private void startServiceThreads() throws IOException {
1410    // Start the executor service pools
1411    this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION, conf.getInt(
1412      HConstants.MASTER_OPEN_REGION_THREADS, HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT));
1413    this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, conf.getInt(
1414      HConstants.MASTER_CLOSE_REGION_THREADS, HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT));
1415    this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
1416      conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,
1417        HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT));
1418    this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
1419      conf.getInt(HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,
1420        HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT));
1421    this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS, conf.getInt(
1422      HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT));
1423    this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, conf.getInt(
1424      SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT));
1425
1426    // We depend on there being only one instance of this executor running
1427    // at a time. To do concurrency, would need fencing of enable/disable of
1428    // tables.
1429    // Any time changing this maxThreads to > 1, pls see the comment at
1430    // AccessController#postCompletedCreateTableAction
1431    this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
1432    startProcedureExecutor();
1433
1434    // Create cleaner thread pool
1435    cleanerPool = new DirScanPool(conf);
1436    // Start log cleaner thread
1437    int cleanerInterval =
1438      conf.getInt(HBASE_MASTER_CLEANER_INTERVAL, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL);
1439    this.logCleaner = new LogCleaner(cleanerInterval, this, conf,
1440      getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(), cleanerPool);
1441    getChoreService().scheduleChore(logCleaner);
1442
1443    // start the hfile archive cleaner thread
1444    Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1445    Map<String, Object> params = new HashMap<>();
1446    params.put(MASTER, this);
1447    this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf,
1448      getMasterFileSystem().getFileSystem(), archiveDir, cleanerPool, params);
1449    getChoreService().scheduleChore(hfileCleaner);
1450
1451    // Regions Reopen based on very high storeFileRefCount is considered enabled
1452    // only if hbase.regions.recovery.store.file.ref.count has value > 0
1453    final int maxStoreFileRefCount = conf.getInt(
1454      HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
1455      HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);
1456    if (maxStoreFileRefCount > 0) {
1457      this.regionsRecoveryChore = new RegionsRecoveryChore(this, conf, this);
1458      getChoreService().scheduleChore(this.regionsRecoveryChore);
1459    } else {
1460      LOG.info("Reopening regions with very high storeFileRefCount is disabled. " +
1461          "Provide threshold value > 0 for {} to enable it.",
1462        HConstants.STORE_FILE_REF_COUNT_THRESHOLD);
1463    }
1464
1465    this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this);
1466
1467    replicationBarrierCleaner = new ReplicationBarrierCleaner(conf, this, getConnection(),
1468      replicationPeerManager);
1469    getChoreService().scheduleChore(replicationBarrierCleaner);
1470
1471    final boolean isSnapshotChoreEnabled = this.snapshotCleanupTracker
1472        .isSnapshotCleanupEnabled();
1473    this.snapshotCleanerChore = new SnapshotCleanerChore(this, conf, getSnapshotManager());
1474    if (isSnapshotChoreEnabled) {
1475      getChoreService().scheduleChore(this.snapshotCleanerChore);
1476    } else {
1477      if (LOG.isTraceEnabled()) {
1478        LOG.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");
1479      }
1480    }
1481    serviceStarted = true;
1482    if (LOG.isTraceEnabled()) {
1483      LOG.trace("Started service threads");
1484    }
1485  }
1486
1487  @Override
1488  protected void stopServiceThreads() {
1489    if (masterJettyServer != null) {
1490      LOG.info("Stopping master jetty server");
1491      try {
1492        masterJettyServer.stop();
1493      } catch (Exception e) {
1494        LOG.error("Failed to stop master jetty server", e);
1495      }
1496    }
1497    stopChores();
1498    if (this.mobCompactThread != null) {
1499      this.mobCompactThread.close();
1500    }
1501    super.stopServiceThreads();
1502    if (cleanerPool != null) {
1503      cleanerPool.shutdownNow();
1504      cleanerPool = null;
1505    }
1506
1507    LOG.debug("Stopping service threads");
1508
1509    // stop procedure executor prior to other services such as server manager and assignment
1510    // manager, as these services are important for some running procedures. See HBASE-24117 for
1511    // example.
1512    stopProcedureExecutor();
1513
1514    if (this.quotaManager != null) {
1515      this.quotaManager.stop();
1516    }
1517
1518    if (this.activeMasterManager != null) {
1519      this.activeMasterManager.stop();
1520    }
1521    if (this.serverManager != null) {
1522      this.serverManager.stop();
1523    }
1524    if (this.assignmentManager != null) {
1525      this.assignmentManager.stop();
1526    }
1527
1528    if (masterRegion != null) {
1529      masterRegion.close(isAborted());
1530    }
1531    if (this.walManager != null) {
1532      this.walManager.stop();
1533    }
1534    if (this.fileSystemManager != null) {
1535      this.fileSystemManager.stop();
1536    }
1537    if (this.mpmHost != null) {
1538      this.mpmHost.stop("server shutting down.");
1539    }
1540    if (this.regionServerTracker != null) {
1541      this.regionServerTracker.stop();
1542    }
1543  }
1544
1545  private void createProcedureExecutor() throws IOException {
1546    MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
1547    procedureStore =
1548      new RegionProcedureStore(this, masterRegion, new MasterProcedureEnv.FsUtilsLeaseRecovery(this));
1549    procedureStore.registerListener(new ProcedureStoreListener() {
1550
1551      @Override
1552      public void abortProcess() {
1553        abort("The Procedure Store lost the lease", null);
1554      }
1555    });
1556    MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
1557    procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
1558    configurationManager.registerObserver(procEnv);
1559
1560    int cpus = Runtime.getRuntime().availableProcessors();
1561    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(
1562      (cpus > 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
1563    final boolean abortOnCorruption =
1564      conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
1565        MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
1566    procedureStore.start(numThreads);
1567    // Just initialize it but do not start the workers, we will start the workers later by calling
1568    // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
1569    // details.
1570    procedureExecutor.init(numThreads, abortOnCorruption);
1571    if (!procEnv.getRemoteDispatcher().start()) {
1572      throw new HBaseIOException("Failed start of remote dispatcher");
1573    }
1574  }
1575
1576  private void startProcedureExecutor() throws IOException {
1577    procedureExecutor.startWorkers();
1578  }
1579
1580  /**
1581   * Turn on/off Snapshot Cleanup Chore
1582   *
1583   * @param on indicates whether Snapshot Cleanup Chore is to be run
1584   */
1585  void switchSnapshotCleanup(final boolean on, final boolean synchronous) {
1586    if (synchronous) {
1587      synchronized (this.snapshotCleanerChore) {
1588        switchSnapshotCleanup(on);
1589      }
1590    } else {
1591      switchSnapshotCleanup(on);
1592    }
1593  }
1594
1595  private void switchSnapshotCleanup(final boolean on) {
1596    try {
1597      snapshotCleanupTracker.setSnapshotCleanupEnabled(on);
1598      if (on) {
1599        if (!getChoreService().isChoreScheduled(this.snapshotCleanerChore)) {
1600          getChoreService().scheduleChore(this.snapshotCleanerChore);
1601        }
1602      } else {
1603        getChoreService().cancelChore(this.snapshotCleanerChore);
1604      }
1605    } catch (KeeperException e) {
1606      LOG.error("Error updating snapshot cleanup mode to {}", on, e);
1607    }
1608  }
1609
1610
1611  private void stopProcedureExecutor() {
1612    if (procedureExecutor != null) {
1613      configurationManager.deregisterObserver(procedureExecutor.getEnvironment());
1614      procedureExecutor.getEnvironment().getRemoteDispatcher().stop();
1615      procedureExecutor.stop();
1616      procedureExecutor.join();
1617      procedureExecutor = null;
1618    }
1619
1620    if (procedureStore != null) {
1621      procedureStore.stop(isAborted());
1622      procedureStore = null;
1623    }
1624  }
1625
1626  private void stopChores() {
1627    ChoreService choreService = getChoreService();
1628    if (choreService != null) {
1629      choreService.cancelChore(this.expiredMobFileCleanerChore);
1630      choreService.cancelChore(this.mobCompactChore);
1631      choreService.cancelChore(this.balancerChore);
1632      choreService.cancelChore(this.normalizerChore);
1633      choreService.cancelChore(this.clusterStatusChore);
1634      choreService.cancelChore(this.catalogJanitorChore);
1635      choreService.cancelChore(this.clusterStatusPublisherChore);
1636      choreService.cancelChore(this.snapshotQuotaChore);
1637      choreService.cancelChore(this.logCleaner);
1638      choreService.cancelChore(this.hfileCleaner);
1639      choreService.cancelChore(this.replicationBarrierCleaner);
1640      choreService.cancelChore(this.snapshotCleanerChore);
1641      choreService.cancelChore(this.hbckChore);
1642      choreService.cancelChore(this.regionsRecoveryChore);
1643    }
1644  }
1645
1646  /**
1647   * @return Get remote side's InetAddress
1648   */
1649  InetAddress getRemoteInetAddress(final int port,
1650      final long serverStartCode) throws UnknownHostException {
1651    // Do it out here in its own little method so can fake an address when
1652    // mocking up in tests.
1653    InetAddress ia = RpcServer.getRemoteIp();
1654
1655    // The call could be from the local regionserver,
1656    // in which case, there is no remote address.
1657    if (ia == null && serverStartCode == startcode) {
1658      InetSocketAddress isa = rpcServices.getSocketAddress();
1659      if (isa != null && isa.getPort() == port) {
1660        ia = isa.getAddress();
1661      }
1662    }
1663    return ia;
1664  }
1665
1666  /**
1667   * @return Maximum time we should run balancer for
1668   */
1669  private int getMaxBalancingTime() {
1670    // if max balancing time isn't set, defaulting it to period time
1671    int maxBalancingTime = getConfiguration().getInt(HConstants.HBASE_BALANCER_MAX_BALANCING,
1672      getConfiguration()
1673        .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));
1674    return maxBalancingTime;
1675  }
1676
1677  /**
1678   * @return Maximum number of regions in transition
1679   */
1680  private int getMaxRegionsInTransition() {
1681    int numRegions = this.assignmentManager.getRegionStates().getRegionAssignments().size();
1682    return Math.max((int) Math.floor(numRegions * this.maxRitPercent), 1);
1683  }
1684
1685  /**
1686   * It first sleep to the next balance plan start time. Meanwhile, throttling by the max
1687   * number regions in transition to protect availability.
1688   * @param nextBalanceStartTime The next balance plan start time
1689   * @param maxRegionsInTransition max number of regions in transition
1690   * @param cutoffTime when to exit balancer
1691   */
1692  private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition,
1693      long cutoffTime) {
1694    boolean interrupted = false;
1695
1696    // Sleep to next balance plan start time
1697    // But if there are zero regions in transition, it can skip sleep to speed up.
1698    while (!interrupted && System.currentTimeMillis() < nextBalanceStartTime
1699        && this.assignmentManager.getRegionStates().hasRegionsInTransition()) {
1700      try {
1701        Thread.sleep(100);
1702      } catch (InterruptedException ie) {
1703        interrupted = true;
1704      }
1705    }
1706
1707    // Throttling by max number regions in transition
1708    while (!interrupted
1709        && maxRegionsInTransition > 0
1710        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
1711        >= maxRegionsInTransition && System.currentTimeMillis() <= cutoffTime) {
1712      try {
1713        // sleep if the number of regions in transition exceeds the limit
1714        Thread.sleep(100);
1715      } catch (InterruptedException ie) {
1716        interrupted = true;
1717      }
1718    }
1719
1720    if (interrupted) Thread.currentThread().interrupt();
1721  }
1722
1723  public boolean balance() throws IOException {
1724    return balance(false);
1725  }
1726
1727  /**
1728   * Checks master state before initiating action over region topology.
1729   * @param action the name of the action under consideration, for logging.
1730   * @return {@code true} when the caller should exit early, {@code false} otherwise.
1731   */
1732  private boolean skipRegionManagementAction(final String action) {
1733    if (!isInitialized()) {
1734      LOG.debug("Master has not been initialized, don't run {}.", action);
1735      return true;
1736    }
1737    if (this.getServerManager().isClusterShutdown()) {
1738      LOG.info("Cluster is shutting down, don't run {}.", action);
1739      return true;
1740    }
1741    if (isInMaintenanceMode()) {
1742      LOG.info("Master is in maintenance mode, don't run {}.", action);
1743      return true;
1744    }
1745    return false;
1746  }
1747
1748  public boolean balance(boolean force) throws IOException {
1749    if (loadBalancerTracker == null || !loadBalancerTracker.isBalancerOn()) {
1750      return false;
1751    }
1752    if (skipRegionManagementAction("balancer")) {
1753      return false;
1754    }
1755
1756    synchronized (this.balancer) {
1757        // Only allow one balance run at at time.
1758      if (this.assignmentManager.hasRegionsInTransition()) {
1759        List<RegionStateNode> regionsInTransition = assignmentManager.getRegionsInTransition();
1760        // if hbase:meta region is in transition, result of assignment cannot be recorded
1761        // ignore the force flag in that case
1762        boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
1763        String prefix = force && !metaInTransition ? "R" : "Not r";
1764        List<RegionStateNode> toPrint = regionsInTransition;
1765        int max = 5;
1766        boolean truncated = false;
1767        if (regionsInTransition.size() > max) {
1768          toPrint = regionsInTransition.subList(0, max);
1769          truncated = true;
1770        }
1771        LOG.info(prefix + "unning balancer because " + regionsInTransition.size() +
1772          " region(s) in transition: " + toPrint + (truncated? "(truncated list)": ""));
1773        if (!force || metaInTransition) return false;
1774      }
1775      if (this.serverManager.areDeadServersInProgress()) {
1776        LOG.info("Not running balancer because processing dead regionserver(s): " +
1777          this.serverManager.getDeadServers());
1778        return false;
1779      }
1780
1781      if (this.cpHost != null) {
1782        try {
1783          if (this.cpHost.preBalance()) {
1784            LOG.debug("Coprocessor bypassing balancer request");
1785            return false;
1786          }
1787        } catch (IOException ioe) {
1788          LOG.error("Error invoking master coprocessor preBalance()", ioe);
1789          return false;
1790        }
1791      }
1792
1793      Map<TableName, Map<ServerName, List<RegionInfo>>> assignments =
1794        this.assignmentManager.getRegionStates()
1795          .getAssignmentsForBalancer(tableStateManager, this.serverManager.getOnlineServersList());
1796      for (Map<ServerName, List<RegionInfo>> serverMap : assignments.values()) {
1797        serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
1798      }
1799
1800      //Give the balancer the current cluster state.
1801      this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
1802
1803      List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
1804
1805      if (skipRegionManagementAction("balancer")) {
1806        // make one last check that the cluster isn't shutting down before proceeding.
1807        return false;
1808      }
1809
1810      List<RegionPlan> sucRPs = executeRegionPlansWithThrottling(plans);
1811
1812      if (this.cpHost != null) {
1813        try {
1814          this.cpHost.postBalance(sucRPs);
1815        } catch (IOException ioe) {
1816          // balancing already succeeded so don't change the result
1817          LOG.error("Error invoking master coprocessor postBalance()", ioe);
1818        }
1819      }
1820    }
1821    // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
1822    // Return true indicating a success.
1823    return true;
1824  }
1825
1826  /**
1827   * Execute region plans with throttling
1828   * @param plans to execute
1829   * @return succeeded plans
1830   */
1831  public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans) {
1832    List<RegionPlan> successRegionPlans = new ArrayList<>();
1833    int maxRegionsInTransition = getMaxRegionsInTransition();
1834    long balanceStartTime = System.currentTimeMillis();
1835    long cutoffTime = balanceStartTime + this.maxBalancingTime;
1836    int rpCount = 0;  // number of RegionPlans balanced so far
1837    if (plans != null && !plans.isEmpty()) {
1838      int balanceInterval = this.maxBalancingTime / plans.size();
1839      LOG.info("Balancer plans size is " + plans.size() + ", the balance interval is "
1840          + balanceInterval + " ms, and the max number regions in transition is "
1841          + maxRegionsInTransition);
1842
1843      for (RegionPlan plan: plans) {
1844        LOG.info("balance " + plan);
1845        //TODO: bulk assign
1846        try {
1847          this.assignmentManager.moveAsync(plan);
1848        } catch (HBaseIOException hioe) {
1849          //should ignore failed plans here, avoiding the whole balance plans be aborted
1850          //later calls of balance() can fetch up the failed and skipped plans
1851          LOG.warn("Failed balance plan {}, skipping...", plan, hioe);
1852        }
1853        //rpCount records balance plans processed, does not care if a plan succeeds
1854        rpCount++;
1855        successRegionPlans.add(plan);
1856
1857        if (this.maxBalancingTime > 0) {
1858          balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
1859            cutoffTime);
1860        }
1861
1862        // if performing next balance exceeds cutoff time, exit the loop
1863        if (this.maxBalancingTime > 0 && rpCount < plans.size()
1864          && System.currentTimeMillis() > cutoffTime) {
1865          // TODO: After balance, there should not be a cutoff time (keeping it as
1866          // a security net for now)
1867          LOG.debug("No more balancing till next balance run; maxBalanceTime="
1868              + this.maxBalancingTime);
1869          break;
1870        }
1871      }
1872    }
1873    return successRegionPlans;
1874  }
1875
1876  @Override
1877  public RegionNormalizer getRegionNormalizer() {
1878    return this.normalizer;
1879  }
1880
1881  /**
1882   * Perform normalization of cluster (invoked by {@link RegionNormalizerChore}).
1883   *
1884   * @return true if an existing normalization was already in progress, or if a new normalization
1885   *   was performed successfully; false otherwise (specifically, if HMaster finished initializing
1886   *   or normalization is globally disabled).
1887   */
1888  public boolean normalizeRegions() throws IOException {
1889    if (regionNormalizerTracker == null || !regionNormalizerTracker.isNormalizerOn()) {
1890      LOG.debug("Region normalization is disabled, don't run region normalizer.");
1891      return false;
1892    }
1893    if (skipRegionManagementAction("region normalizer")) {
1894      return false;
1895    }
1896    if (assignmentManager.hasRegionsInTransition()) {
1897      return false;
1898    }
1899
1900    if (!normalizationInProgressLock.tryLock()) {
1901      // Don't run the normalizer concurrently
1902      LOG.info("Normalization already in progress. Skipping request.");
1903      return true;
1904    }
1905
1906    try {
1907      final List<TableName> allEnabledTables =
1908        new ArrayList<>(tableStateManager.getTablesInStates(TableState.State.ENABLED));
1909      Collections.shuffle(allEnabledTables);
1910
1911      final List<Long> submittedPlanProcIds = new ArrayList<>();
1912      for (TableName table : allEnabledTables) {
1913        if (table.isSystemTable()) {
1914          continue;
1915        }
1916        final TableDescriptor tblDesc = getTableDescriptors().get(table);
1917        if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
1918          LOG.debug(
1919            "Skipping table {} because normalization is disabled in its table properties.", table);
1920          continue;
1921        }
1922
1923        // make one last check that the cluster isn't shutting down before proceeding.
1924        if (skipRegionManagementAction("region normalizer")) {
1925          return false;
1926        }
1927
1928        final List<NormalizationPlan> plans = normalizer.computePlansForTable(table);
1929        if (CollectionUtils.isEmpty(plans)) {
1930          LOG.debug("No normalization required for table {}.", table);
1931          continue;
1932        }
1933
1934        // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to
1935        // submit task , so there's no artificial rate-
1936        // limiting of merge/split requests due to this serial loop.
1937        for (NormalizationPlan plan : plans) {
1938          long procId = plan.submit(this);
1939          submittedPlanProcIds.add(procId);
1940          if (plan.getType() == PlanType.SPLIT) {
1941            splitPlanCount++;
1942          } else if (plan.getType() == PlanType.MERGE) {
1943            mergePlanCount++;
1944          }
1945        }
1946        int totalPlansSubmitted = submittedPlanProcIds.size();
1947        if (totalPlansSubmitted > 0 && LOG.isDebugEnabled()) {
1948          LOG.debug("Normalizer plans submitted. Total plans count: {} , procID list: {}",
1949            totalPlansSubmitted, submittedPlanProcIds);
1950        }
1951      }
1952    } finally {
1953      normalizationInProgressLock.unlock();
1954    }
1955    return true;
1956  }
1957
1958  /**
1959   * @return Client info for use as prefix on an audit log string; who did an action
1960   */
1961  @Override
1962  public String getClientIdAuditPrefix() {
1963    return "Client=" + RpcServer.getRequestUserName().orElse(null)
1964        + "/" + RpcServer.getRemoteAddress().orElse(null);
1965  }
1966
1967  /**
1968   * Switch for the background CatalogJanitor thread.
1969   * Used for testing.  The thread will continue to run.  It will just be a noop
1970   * if disabled.
1971   * @param b If false, the catalog janitor won't do anything.
1972   */
1973  public void setCatalogJanitorEnabled(final boolean b) {
1974    this.catalogJanitorChore.setEnabled(b);
1975  }
1976
1977  @Override
1978  public long mergeRegions(
1979      final RegionInfo[] regionsToMerge,
1980      final boolean forcible,
1981      final long ng,
1982      final long nonce) throws IOException {
1983    checkInitialized();
1984
1985    if (!isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
1986      String regionsStr = Arrays.deepToString(regionsToMerge);
1987      LOG.warn("Merge switch is off! skip merge of " + regionsStr);
1988      throw new DoNotRetryIOException("Merge of " + regionsStr +
1989          " failed because merge switch is off");
1990    }
1991
1992    final String mergeRegionsStr = Arrays.stream(regionsToMerge).map(RegionInfo::getEncodedName)
1993      .collect(Collectors.joining(", "));
1994    return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {
1995      @Override
1996      protected void run() throws IOException {
1997        getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
1998        String aid = getClientIdAuditPrefix();
1999        LOG.info("{} merge regions {}", aid, mergeRegionsStr);
2000        submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
2001            regionsToMerge, forcible));
2002        getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
2003      }
2004
2005      @Override
2006      protected String getDescription() {
2007        return "MergeTableProcedure";
2008      }
2009    });
2010  }
2011
2012  @Override
2013  public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow,
2014      final long nonceGroup, final long nonce)
2015  throws IOException {
2016    checkInitialized();
2017
2018    if (!isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
2019      LOG.warn("Split switch is off! skip split of " + regionInfo);
2020      throw new DoNotRetryIOException("Split region " + regionInfo.getRegionNameAsString() +
2021          " failed due to split switch off");
2022    }
2023
2024    return MasterProcedureUtil.submitProcedure(
2025        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2026      @Override
2027      protected void run() throws IOException {
2028        getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
2029        LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());
2030
2031        // Execute the operation asynchronously
2032        submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
2033      }
2034
2035      @Override
2036      protected String getDescription() {
2037        return "SplitTableProcedure";
2038      }
2039    });
2040  }
2041
2042  // Public so can be accessed by tests. Blocks until move is done.
2043  // Replace with an async implementation from which you can get
2044  // a success/failure result.
2045  @VisibleForTesting
2046  public void move(final byte[] encodedRegionName, byte[] destServerName) throws HBaseIOException {
2047    RegionState regionState = assignmentManager.getRegionStates().
2048      getRegionState(Bytes.toString(encodedRegionName));
2049
2050    RegionInfo hri;
2051    if (regionState != null) {
2052      hri = regionState.getRegion();
2053    } else {
2054      throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
2055    }
2056
2057    ServerName dest;
2058    List<ServerName> exclude = hri.getTable().isSystemTable() ? assignmentManager.getExcludedServersForSystemTable()
2059        : new ArrayList<>(1);
2060    if (destServerName != null && exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))) {
2061      LOG.info(
2062          Bytes.toString(encodedRegionName) + " can not move to " + Bytes.toString(destServerName)
2063              + " because the server is in exclude list");
2064      destServerName = null;
2065    }
2066    if (destServerName == null || destServerName.length == 0) {
2067      LOG.info("Passed destination servername is null/empty so " +
2068        "choosing a server at random");
2069      exclude.add(regionState.getServerName());
2070      final List<ServerName> destServers = this.serverManager.createDestinationServersList(exclude);
2071      dest = balancer.randomAssignment(hri, destServers);
2072      if (dest == null) {
2073        LOG.debug("Unable to determine a plan to assign " + hri);
2074        return;
2075      }
2076    } else {
2077      ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
2078      dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
2079      if (dest == null) {
2080        LOG.debug("Unable to determine a plan to assign " + hri);
2081        return;
2082      }
2083      // TODO: What is this? I don't get it.
2084      if (dest.equals(serverName) && balancer instanceof BaseLoadBalancer
2085          && !((BaseLoadBalancer)balancer).shouldBeOnMaster(hri)) {
2086        // To avoid unnecessary region moving later by balancer. Don't put user
2087        // regions on master.
2088        LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2089          + " to avoid unnecessary region moving later by load balancer,"
2090          + " because it should not be on master");
2091        return;
2092      }
2093    }
2094
2095    if (dest.equals(regionState.getServerName())) {
2096      LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2097        + " because region already assigned to the same server " + dest + ".");
2098      return;
2099    }
2100
2101    // Now we can do the move
2102    RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
2103    assert rp.getDestination() != null: rp.toString() + " " + dest;
2104
2105    try {
2106      checkInitialized();
2107      if (this.cpHost != null) {
2108        this.cpHost.preMove(hri, rp.getSource(), rp.getDestination());
2109      }
2110
2111      TransitRegionStateProcedure proc =
2112          this.assignmentManager.createMoveRegionProcedure(rp.getRegionInfo(), rp.getDestination());
2113      // Warmup the region on the destination before initiating the move. this call
2114      // is synchronous and takes some time. doing it before the source region gets
2115      // closed
2116      serverManager.sendRegionWarmup(rp.getDestination(), hri);
2117      LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
2118      Future<byte[]> future = ProcedureSyncWait.submitProcedure(this.procedureExecutor, proc);
2119      try {
2120        // Is this going to work? Will we throw exception on error?
2121        // TODO: CompletableFuture rather than this stunted Future.
2122        future.get();
2123      } catch (InterruptedException | ExecutionException e) {
2124        throw new HBaseIOException(e);
2125      }
2126      if (this.cpHost != null) {
2127        this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
2128      }
2129    } catch (IOException ioe) {
2130      if (ioe instanceof HBaseIOException) {
2131        throw (HBaseIOException)ioe;
2132      }
2133      throw new HBaseIOException(ioe);
2134    }
2135  }
2136
2137  @Override
2138  public long createTable(final TableDescriptor tableDescriptor, final byte[][] splitKeys,
2139      final long nonceGroup, final long nonce) throws IOException {
2140    checkInitialized();
2141    TableDescriptor desc = getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor);
2142    if (desc == null) {
2143      throw new IOException("Creation for " + tableDescriptor + " is canceled by CP");
2144    }
2145    String namespace = desc.getTableName().getNamespaceAsString();
2146    this.clusterSchemaService.getNamespace(namespace);
2147
2148    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(desc, splitKeys);
2149    TableDescriptorChecker.sanityCheck(conf, desc);
2150
2151    return MasterProcedureUtil
2152      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2153        @Override
2154        protected void run() throws IOException {
2155          getMaster().getMasterCoprocessorHost().preCreateTable(desc, newRegions);
2156
2157          LOG.info(getClientIdAuditPrefix() + " create " + desc);
2158
2159          // TODO: We can handle/merge duplicate requests, and differentiate the case of
2160          // TableExistsException by saying if the schema is the same or not.
2161          //
2162          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2163          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2164          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2165          submitProcedure(
2166            new CreateTableProcedure(procedureExecutor.getEnvironment(), desc, newRegions, latch));
2167          latch.await();
2168
2169          getMaster().getMasterCoprocessorHost().postCreateTable(desc, newRegions);
2170        }
2171
2172        @Override
2173        protected String getDescription() {
2174          return "CreateTableProcedure";
2175        }
2176      });
2177  }
2178
2179  @Override
2180  public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
2181    if (isStopped()) {
2182      throw new MasterNotRunningException();
2183    }
2184
2185    TableName tableName = tableDescriptor.getTableName();
2186    if (!(tableName.isSystemTable())) {
2187      throw new IllegalArgumentException(
2188        "Only system table creation can use this createSystemTable API");
2189    }
2190
2191    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
2192
2193    LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
2194
2195    // This special create table is called locally to master.  Therefore, no RPC means no need
2196    // to use nonce to detect duplicated RPC call.
2197    long procId = this.procedureExecutor.submitProcedure(
2198      new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
2199
2200    return procId;
2201  }
2202
2203  private void startActiveMasterManager(int infoPort) throws KeeperException {
2204    String backupZNode = ZNodePaths.joinZNode(
2205      zooKeeper.getZNodePaths().backupMasterAddressesZNode, serverName.toString());
2206    /*
2207    * Add a ZNode for ourselves in the backup master directory since we
2208    * may not become the active master. If so, we want the actual active
2209    * master to know we are backup masters, so that it won't assign
2210    * regions to us if so configured.
2211    *
2212    * If we become the active master later, ActiveMasterManager will delete
2213    * this node explicitly.  If we crash before then, ZooKeeper will delete
2214    * this node for us since it is ephemeral.
2215    */
2216    LOG.info("Adding backup master ZNode " + backupZNode);
2217    if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode, serverName, infoPort)) {
2218      LOG.warn("Failed create of " + backupZNode + " by " + serverName);
2219    }
2220    this.activeMasterManager.setInfoPort(infoPort);
2221    int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
2222    // If we're a backup master, stall until a primary to write this address
2223    if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
2224      LOG.debug("HMaster started in backup mode. Stalling until master znode is written.");
2225      // This will only be a minute or so while the cluster starts up,
2226      // so don't worry about setting watches on the parent znode
2227      while (!activeMasterManager.hasActiveMaster()) {
2228        LOG.debug("Waiting for master address and cluster state znode to be written.");
2229        Threads.sleep(timeout);
2230      }
2231    }
2232    MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2233    status.setDescription("Master startup");
2234    try {
2235      if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2236        finishActiveMasterInitialization(status);
2237      }
2238    } catch (Throwable t) {
2239      status.setStatus("Failed to become active: " + t.getMessage());
2240      LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
2241      // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2242      if (t instanceof NoClassDefFoundError && t.getMessage().
2243          contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) {
2244        // improved error message for this special case
2245        abort("HBase is having a problem with its Hadoop jars.  You may need to recompile " +
2246          "HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion() +
2247          " or change your hadoop jars to start properly", t);
2248      } else {
2249        abort("Unhandled exception. Starting shutdown.", t);
2250      }
2251    } finally {
2252      status.cleanup();
2253    }
2254  }
2255
2256  private static boolean isCatalogTable(final TableName tableName) {
2257    return tableName.equals(TableName.META_TABLE_NAME);
2258  }
2259
2260  @Override
2261  public long deleteTable(
2262      final TableName tableName,
2263      final long nonceGroup,
2264      final long nonce) throws IOException {
2265    checkInitialized();
2266
2267    return MasterProcedureUtil.submitProcedure(
2268        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2269      @Override
2270      protected void run() throws IOException {
2271        getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);
2272
2273        LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
2274
2275        // TODO: We can handle/merge duplicate request
2276        //
2277        // We need to wait for the procedure to potentially fail due to "prepare" sanity
2278        // checks. This will block only the beginning of the procedure. See HBASE-19953.
2279        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2280        submitProcedure(new DeleteTableProcedure(procedureExecutor.getEnvironment(),
2281            tableName, latch));
2282        latch.await();
2283
2284        getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);
2285      }
2286
2287      @Override
2288      protected String getDescription() {
2289        return "DeleteTableProcedure";
2290      }
2291    });
2292  }
2293
2294  @Override
2295  public long truncateTable(
2296      final TableName tableName,
2297      final boolean preserveSplits,
2298      final long nonceGroup,
2299      final long nonce) throws IOException {
2300    checkInitialized();
2301
2302    return MasterProcedureUtil.submitProcedure(
2303        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2304      @Override
2305      protected void run() throws IOException {
2306        getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);
2307
2308        LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
2309        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
2310        submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(),
2311            tableName, preserveSplits, latch));
2312        latch.await();
2313
2314        getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);
2315      }
2316
2317      @Override
2318      protected String getDescription() {
2319        return "TruncateTableProcedure";
2320      }
2321    });
2322  }
2323
2324  @Override
2325  public long addColumn(final TableName tableName, final ColumnFamilyDescriptor column,
2326      final long nonceGroup, final long nonce) throws IOException {
2327    checkInitialized();
2328    checkTableExists(tableName);
2329
2330    return modifyTable(tableName, new TableDescriptorGetter() {
2331
2332      @Override
2333      public TableDescriptor get() throws IOException {
2334        TableDescriptor old = getTableDescriptors().get(tableName);
2335        if (old.hasColumnFamily(column.getName())) {
2336          throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()
2337              + "' in table '" + tableName + "' already exists so cannot be added");
2338        }
2339
2340        return TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();
2341      }
2342    }, nonceGroup, nonce, true);
2343  }
2344
2345  /**
2346   * Implement to return TableDescriptor after pre-checks
2347   */
2348  protected interface TableDescriptorGetter {
2349    TableDescriptor get() throws IOException;
2350  }
2351
2352  @Override
2353  public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,
2354      final long nonceGroup, final long nonce) throws IOException {
2355    checkInitialized();
2356    checkTableExists(tableName);
2357    return modifyTable(tableName, new TableDescriptorGetter() {
2358
2359      @Override
2360      public TableDescriptor get() throws IOException {
2361        TableDescriptor old = getTableDescriptors().get(tableName);
2362        if (!old.hasColumnFamily(descriptor.getName())) {
2363          throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()
2364              + "' does not exist, so it cannot be modified");
2365        }
2366
2367        return TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();
2368      }
2369    }, nonceGroup, nonce, true);
2370  }
2371
2372  @Override
2373  public long deleteColumn(final TableName tableName, final byte[] columnName,
2374      final long nonceGroup, final long nonce) throws IOException {
2375    checkInitialized();
2376    checkTableExists(tableName);
2377
2378    return modifyTable(tableName, new TableDescriptorGetter() {
2379
2380      @Override
2381      public TableDescriptor get() throws IOException {
2382        TableDescriptor old = getTableDescriptors().get(tableName);
2383
2384        if (!old.hasColumnFamily(columnName)) {
2385          throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
2386              + "' does not exist, so it cannot be deleted");
2387        }
2388        if (old.getColumnFamilyCount() == 1) {
2389          throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
2390              + "' is the only column family in the table, so it cannot be deleted");
2391        }
2392        return TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();
2393      }
2394    }, nonceGroup, nonce, true);
2395  }
2396
2397  @Override
2398  public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
2399      throws IOException {
2400    checkInitialized();
2401
2402    return MasterProcedureUtil.submitProcedure(
2403        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2404      @Override
2405      protected void run() throws IOException {
2406        getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
2407
2408        // Normally, it would make sense for this authorization check to exist inside
2409        // AccessController, but because the authorization check is done based on internal state
2410        // (rather than explicit permissions) we'll do the check here instead of in the
2411        // coprocessor.
2412        MasterQuotaManager quotaManager = getMasterQuotaManager();
2413        if (quotaManager != null) {
2414          if (quotaManager.isQuotaInitialized()) {
2415              SpaceQuotaSnapshot currSnapshotOfTable =
2416                  QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);
2417              if (currSnapshotOfTable != null) {
2418                SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
2419                if (quotaStatus.isInViolation()
2420                    && SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)) {
2421                throw new AccessDeniedException("Enabling the table '" + tableName
2422                    + "' is disallowed due to a violated space quota.");
2423              }
2424            }
2425          } else if (LOG.isTraceEnabled()) {
2426            LOG.trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
2427          }
2428        }
2429
2430        LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
2431
2432        // Execute the operation asynchronously - client will check the progress of the operation
2433        // In case the request is from a <1.1 client before returning,
2434        // we want to make sure that the table is prepared to be
2435        // enabled (the table is locked and the table state is set).
2436        // Note: if the procedure throws exception, we will catch it and rethrow.
2437        final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
2438        submitProcedure(new EnableTableProcedure(procedureExecutor.getEnvironment(),
2439            tableName, prepareLatch));
2440        prepareLatch.await();
2441
2442        getMaster().getMasterCoprocessorHost().postEnableTable(tableName);
2443      }
2444
2445      @Override
2446      protected String getDescription() {
2447        return "EnableTableProcedure";
2448      }
2449    });
2450  }
2451
2452  @Override
2453  public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
2454      throws IOException {
2455    checkInitialized();
2456
2457    return MasterProcedureUtil.submitProcedure(
2458        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2459      @Override
2460      protected void run() throws IOException {
2461        getMaster().getMasterCoprocessorHost().preDisableTable(tableName);
2462
2463        LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
2464
2465        // Execute the operation asynchronously - client will check the progress of the operation
2466        // In case the request is from a <1.1 client before returning,
2467        // we want to make sure that the table is prepared to be
2468        // enabled (the table is locked and the table state is set).
2469        // Note: if the procedure throws exception, we will catch it and rethrow.
2470        //
2471        // We need to wait for the procedure to potentially fail due to "prepare" sanity
2472        // checks. This will block only the beginning of the procedure. See HBASE-19953.
2473        final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();
2474        submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(),
2475            tableName, false, prepareLatch));
2476        prepareLatch.await();
2477
2478        getMaster().getMasterCoprocessorHost().postDisableTable(tableName);
2479      }
2480
2481      @Override
2482      protected String getDescription() {
2483        return "DisableTableProcedure";
2484      }
2485    });
2486  }
2487
2488  private long modifyTable(final TableName tableName,
2489      final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,
2490      final boolean shouldCheckDescriptor) throws IOException {
2491    return MasterProcedureUtil
2492        .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2493          @Override
2494          protected void run() throws IOException {
2495            TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName);
2496            TableDescriptor newDescriptor = getMaster().getMasterCoprocessorHost()
2497                .preModifyTable(tableName, oldDescriptor, newDescriptorGetter.get());
2498            TableDescriptorChecker.sanityCheck(conf, newDescriptor);
2499            LOG.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName,
2500              oldDescriptor, newDescriptor);
2501
2502            // Execute the operation synchronously - wait for the operation completes before
2503            // continuing.
2504            //
2505            // We need to wait for the procedure to potentially fail due to "prepare" sanity
2506            // checks. This will block only the beginning of the procedure. See HBASE-19953.
2507            ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2508            submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),
2509                newDescriptor, latch, oldDescriptor, shouldCheckDescriptor));
2510            latch.await();
2511
2512            getMaster().getMasterCoprocessorHost().postModifyTable(tableName, oldDescriptor,
2513              newDescriptor);
2514          }
2515
2516          @Override
2517          protected String getDescription() {
2518            return "ModifyTableProcedure";
2519          }
2520        });
2521
2522  }
2523
2524  @Override
2525  public long modifyTable(final TableName tableName, final TableDescriptor newDescriptor,
2526      final long nonceGroup, final long nonce) throws IOException {
2527    checkInitialized();
2528    return modifyTable(tableName, new TableDescriptorGetter() {
2529      @Override
2530      public TableDescriptor get() throws IOException {
2531        return newDescriptor;
2532      }
2533    }, nonceGroup, nonce, false);
2534
2535  }
2536
2537  public long restoreSnapshot(final SnapshotDescription snapshotDesc,
2538      final long nonceGroup, final long nonce, final boolean restoreAcl) throws IOException {
2539    checkInitialized();
2540    getSnapshotManager().checkSnapshotSupport();
2541
2542    // Ensure namespace exists. Will throw exception if non-known NS.
2543    final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());
2544    getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
2545
2546    return MasterProcedureUtil.submitProcedure(
2547        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2548      @Override
2549      protected void run() throws IOException {
2550          setProcId(
2551            getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl));
2552      }
2553
2554      @Override
2555      protected String getDescription() {
2556        return "RestoreSnapshotProcedure";
2557      }
2558    });
2559  }
2560
2561  private void checkTableExists(final TableName tableName)
2562      throws IOException, TableNotFoundException {
2563    if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
2564      throw new TableNotFoundException(tableName);
2565    }
2566  }
2567
2568  @Override
2569  public void checkTableModifiable(final TableName tableName)
2570      throws IOException, TableNotFoundException, TableNotDisabledException {
2571    if (isCatalogTable(tableName)) {
2572      throw new IOException("Can't modify catalog tables");
2573    }
2574    checkTableExists(tableName);
2575    TableState ts = getTableStateManager().getTableState(tableName);
2576    if (!ts.isDisabled()) {
2577      throw new TableNotDisabledException("Not DISABLED; " + ts);
2578    }
2579  }
2580
2581  public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
2582    return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
2583  }
2584
2585  public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<Option> options)
2586      throws InterruptedIOException {
2587    ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
2588    // given that hbase1 can't submit the request with Option,
2589    // we return all information to client if the list of Option is empty.
2590    if (options.isEmpty()) {
2591      options = EnumSet.allOf(Option.class);
2592    }
2593
2594    for (Option opt : options) {
2595      switch (opt) {
2596        case HBASE_VERSION: builder.setHBaseVersion(VersionInfo.getVersion()); break;
2597        case CLUSTER_ID: builder.setClusterId(getClusterId()); break;
2598        case MASTER: builder.setMasterName(getServerName()); break;
2599        case BACKUP_MASTERS: builder.setBackerMasterNames(getBackupMasters()); break;
2600        case LIVE_SERVERS: {
2601          if (serverManager != null) {
2602            builder.setLiveServerMetrics(serverManager.getOnlineServers().entrySet().stream()
2603              .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
2604          }
2605          break;
2606        }
2607        case DEAD_SERVERS: {
2608          if (serverManager != null) {
2609            builder.setDeadServerNames(new ArrayList<>(
2610              serverManager.getDeadServers().copyServerNames()));
2611          }
2612          break;
2613        }
2614        case MASTER_COPROCESSORS: {
2615          if (cpHost != null) {
2616            builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));
2617          }
2618          break;
2619        }
2620        case REGIONS_IN_TRANSITION: {
2621          if (assignmentManager != null) {
2622            builder.setRegionsInTransition(assignmentManager.getRegionStates()
2623                .getRegionsStateInTransition());
2624          }
2625          break;
2626        }
2627        case BALANCER_ON: {
2628          if (loadBalancerTracker != null) {
2629            builder.setBalancerOn(loadBalancerTracker.isBalancerOn());
2630          }
2631          break;
2632        }
2633        case MASTER_INFO_PORT: {
2634          if (infoServer != null) {
2635            builder.setMasterInfoPort(infoServer.getPort());
2636          }
2637          break;
2638        }
2639        case SERVERS_NAME: {
2640          if (serverManager != null) {
2641            builder.setServerNames(serverManager.getOnlineServersList());
2642          }
2643          break;
2644        }
2645        case TABLE_TO_REGIONS_COUNT: {
2646          if (isActiveMaster() && isInitialized() && assignmentManager != null) {
2647            try {
2648              Map<TableName, RegionStatesCount> tableRegionStatesCountMap = new HashMap<>();
2649              Map<String, TableDescriptor> tableDescriptorMap = getTableDescriptors().getAll();
2650              for (TableDescriptor tableDescriptor : tableDescriptorMap.values()) {
2651                TableName tableName = tableDescriptor.getTableName();
2652                RegionStatesCount regionStatesCount = assignmentManager
2653                  .getRegionStatesCount(tableName);
2654                tableRegionStatesCountMap.put(tableName, regionStatesCount);
2655              }
2656              builder.setTableRegionStatesCount(tableRegionStatesCountMap);
2657            } catch (IOException e) {
2658              LOG.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e);
2659            }
2660          }
2661          break;
2662        }
2663      }
2664    }
2665    return builder.build();
2666  }
2667
2668  /**
2669   * @return cluster status
2670   */
2671  public ClusterMetrics getClusterMetrics() throws IOException {
2672    return getClusterMetrics(EnumSet.allOf(Option.class));
2673  }
2674
2675  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
2676    if (cpHost != null) {
2677      cpHost.preGetClusterMetrics();
2678    }
2679    ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);
2680    if (cpHost != null) {
2681      cpHost.postGetClusterMetrics(status);
2682    }
2683    return status;
2684  }
2685
2686  private List<ServerName> getBackupMasters() throws InterruptedIOException {
2687    // Build Set of backup masters from ZK nodes
2688    List<String> backupMasterStrings;
2689    try {
2690      backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
2691        this.zooKeeper.getZNodePaths().backupMasterAddressesZNode);
2692    } catch (KeeperException e) {
2693      LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
2694      backupMasterStrings = null;
2695    }
2696
2697    List<ServerName> backupMasters = Collections.emptyList();
2698    if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
2699      backupMasters = new ArrayList<>(backupMasterStrings.size());
2700      for (String s: backupMasterStrings) {
2701        try {
2702          byte [] bytes;
2703          try {
2704            bytes = ZKUtil.getData(this.zooKeeper, ZNodePaths.joinZNode(
2705                this.zooKeeper.getZNodePaths().backupMasterAddressesZNode, s));
2706          } catch (InterruptedException e) {
2707            throw new InterruptedIOException();
2708          }
2709          if (bytes != null) {
2710            ServerName sn;
2711            try {
2712              sn = ProtobufUtil.parseServerNameFrom(bytes);
2713            } catch (DeserializationException e) {
2714              LOG.warn("Failed parse, skipping registering backup server", e);
2715              continue;
2716            }
2717            backupMasters.add(sn);
2718          }
2719        } catch (KeeperException e) {
2720          LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
2721                   "backup servers"), e);
2722        }
2723      }
2724      Collections.sort(backupMasters, new Comparator<ServerName>() {
2725        @Override
2726        public int compare(ServerName s1, ServerName s2) {
2727          return s1.getServerName().compareTo(s2.getServerName());
2728        }});
2729    }
2730    return backupMasters;
2731  }
2732
2733  /**
2734   * The set of loaded coprocessors is stored in a static set. Since it's
2735   * statically allocated, it does not require that HMaster's cpHost be
2736   * initialized prior to accessing it.
2737   * @return a String representation of the set of names of the loaded coprocessors.
2738   */
2739  public static String getLoadedCoprocessors() {
2740    return CoprocessorHost.getLoadedCoprocessors().toString();
2741  }
2742
2743  /**
2744   * @return timestamp in millis when HMaster was started.
2745   */
2746  public long getMasterStartTime() {
2747    return startcode;
2748  }
2749
2750  /**
2751   * @return timestamp in millis when HMaster became the active master.
2752   */
2753  public long getMasterActiveTime() {
2754    return masterActiveTime;
2755  }
2756
2757  /**
2758   * @return timestamp in millis when HMaster finished becoming the active master
2759   */
2760  public long getMasterFinishedInitializationTime() {
2761    return masterFinishedInitializationTime;
2762  }
2763
2764  public int getNumWALFiles() {
2765    return 0;
2766  }
2767
2768  public ProcedureStore getProcedureStore() {
2769    return procedureStore;
2770  }
2771
2772  public int getRegionServerInfoPort(final ServerName sn) {
2773    int port = this.serverManager.getInfoPort(sn);
2774    return port == 0 ? conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
2775      HConstants.DEFAULT_REGIONSERVER_INFOPORT) : port;
2776  }
2777
2778  @Override
2779  public String getRegionServerVersion(ServerName sn) {
2780    // Will return "0.0.0" if the server is not online to prevent move system region to unknown
2781    // version RS.
2782    return this.serverManager.getVersion(sn);
2783  }
2784
2785  @Override
2786  public void checkIfShouldMoveSystemRegionAsync() {
2787    assignmentManager.checkIfShouldMoveSystemRegionAsync();
2788  }
2789
2790  /**
2791   * @return array of coprocessor SimpleNames.
2792   */
2793  public String[] getMasterCoprocessors() {
2794    Set<String> masterCoprocessors = getMasterCoprocessorHost().getCoprocessors();
2795    return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
2796  }
2797
2798  @Override
2799  public void abort(String reason, Throwable cause) {
2800    if (isAborted() || isStopped()) {
2801      LOG.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
2802      return;
2803    }
2804    setAbortRequested();
2805    if (cpHost != null) {
2806      // HBASE-4014: dump a list of loaded coprocessors.
2807      LOG.error(HBaseMarkers.FATAL, "Master server abort: loaded coprocessors are: " +
2808          getLoadedCoprocessors());
2809    }
2810    String msg = "***** ABORTING master " + this + ": " + reason + " *****";
2811    if (cause != null) {
2812      LOG.error(HBaseMarkers.FATAL, msg, cause);
2813    } else {
2814      LOG.error(HBaseMarkers.FATAL, msg);
2815    }
2816
2817    try {
2818      stopMaster();
2819    } catch (IOException e) {
2820      LOG.error("Exception occurred while stopping master", e);
2821    }
2822  }
2823
2824  @Override
2825  public ZKWatcher getZooKeeper() {
2826    return zooKeeper;
2827  }
2828
2829  @Override
2830  public MasterCoprocessorHost getMasterCoprocessorHost() {
2831    return cpHost;
2832  }
2833
2834  @Override
2835  public MasterQuotaManager getMasterQuotaManager() {
2836    return quotaManager;
2837  }
2838
2839  @Override
2840  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
2841    return procedureExecutor;
2842  }
2843
2844  @Override
2845  public ServerName getServerName() {
2846    return this.serverName;
2847  }
2848
2849  @Override
2850  public AssignmentManager getAssignmentManager() {
2851    return this.assignmentManager;
2852  }
2853
2854  @Override
2855  public CatalogJanitor getCatalogJanitor() {
2856    return this.catalogJanitorChore;
2857  }
2858
2859  public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
2860    return rsFatals;
2861  }
2862
2863  /**
2864   * Shutdown the cluster.
2865   * Master runs a coordinated stop of all RegionServers and then itself.
2866   */
2867  public void shutdown() throws IOException {
2868    if (cpHost != null) {
2869      cpHost.preShutdown();
2870    }
2871
2872    // Tell the servermanager cluster shutdown has been called. This makes it so when Master is
2873    // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
2874    // the cluster status as down. RegionServers will notice this change in state and will start
2875    // shutting themselves down. When last has exited, Master can go down.
2876    if (this.serverManager != null) {
2877      this.serverManager.shutdownCluster();
2878    }
2879    if (this.clusterStatusTracker != null) {
2880      try {
2881        this.clusterStatusTracker.setClusterDown();
2882      } catch (KeeperException e) {
2883        LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
2884      }
2885    }
2886    // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,
2887    // processing so we can go down.
2888    if (this.procedureExecutor != null) {
2889      this.procedureExecutor.stop();
2890    }
2891    // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;
2892    // this is what we want especially if the Master is in startup phase doing call outs to
2893    // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
2894    // the rpc to timeout.
2895    if (this.clusterConnection != null) {
2896      this.clusterConnection.close();
2897    }
2898  }
2899
2900  public void stopMaster() throws IOException {
2901    if (cpHost != null) {
2902      cpHost.preStopMaster();
2903    }
2904    stop("Stopped by " + Thread.currentThread().getName());
2905  }
2906
2907  @Override
2908  public void stop(String msg) {
2909    if (!isStopped()) {
2910      super.stop(msg);
2911      if (this.activeMasterManager != null) {
2912        this.activeMasterManager.stop();
2913      }
2914    }
2915  }
2916
2917  @VisibleForTesting
2918  protected void checkServiceStarted() throws ServerNotRunningYetException {
2919    if (!serviceStarted) {
2920      throw new ServerNotRunningYetException("Server is not running yet");
2921    }
2922  }
2923
2924  public static class MasterStoppedException extends DoNotRetryIOException {
2925    MasterStoppedException() {
2926      super();
2927    }
2928  }
2929
2930  void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException,
2931      MasterNotRunningException, MasterStoppedException {
2932    checkServiceStarted();
2933    if (!isInitialized()) {
2934      throw new PleaseHoldException("Master is initializing");
2935    }
2936    if (isStopped()) {
2937      throw new MasterStoppedException();
2938    }
2939  }
2940
2941  /**
2942   * Report whether this master is currently the active master or not.
2943   * If not active master, we are parked on ZK waiting to become active.
2944   *
2945   * This method is used for testing.
2946   *
2947   * @return true if active master, false if not.
2948   */
2949  @Override
2950  public boolean isActiveMaster() {
2951    return activeMaster;
2952  }
2953
2954  /**
2955   * Report whether this master has completed with its initialization and is
2956   * ready.  If ready, the master is also the active master.  A standby master
2957   * is never ready.
2958   *
2959   * This method is used for testing.
2960   *
2961   * @return true if master is ready to go, false if not.
2962   */
2963  @Override
2964  public boolean isInitialized() {
2965    return initialized.isReady();
2966  }
2967
2968  /**
2969   * Report whether this master is in maintenance mode.
2970   *
2971   * @return true if master is in maintenanceMode
2972   */
2973  @Override
2974  public boolean isInMaintenanceMode() {
2975    return maintenanceMode;
2976  }
2977
2978  @VisibleForTesting
2979  public void setInitialized(boolean isInitialized) {
2980    procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
2981  }
2982
2983  @Override
2984  public ProcedureEvent<?> getInitializedEvent() {
2985    return initialized;
2986  }
2987
2988  /**
2989   * Compute the average load across all region servers.
2990   * Currently, this uses a very naive computation - just uses the number of
2991   * regions being served, ignoring stats about number of requests.
2992   * @return the average load
2993   */
2994  public double getAverageLoad() {
2995    if (this.assignmentManager == null) {
2996      return 0;
2997    }
2998
2999    RegionStates regionStates = this.assignmentManager.getRegionStates();
3000    if (regionStates == null) {
3001      return 0;
3002    }
3003    return regionStates.getAverageLoad();
3004  }
3005
3006  /*
3007   * @return the count of region split plans executed
3008   */
3009  public long getSplitPlanCount() {
3010    return splitPlanCount;
3011  }
3012
3013  /*
3014   * @return the count of region merge plans executed
3015   */
3016  public long getMergePlanCount() {
3017    return mergePlanCount;
3018  }
3019
3020  @Override
3021  public boolean registerService(Service instance) {
3022    /*
3023     * No stacking of instances is allowed for a single service name
3024     */
3025    Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
3026    String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
3027    if (coprocessorServiceHandlers.containsKey(serviceName)) {
3028      LOG.error("Coprocessor service "+serviceName+
3029          " already registered, rejecting request from "+instance
3030      );
3031      return false;
3032    }
3033
3034    coprocessorServiceHandlers.put(serviceName, instance);
3035    if (LOG.isDebugEnabled()) {
3036      LOG.debug("Registered master coprocessor service: service="+serviceName);
3037    }
3038    return true;
3039  }
3040
3041  /**
3042   * Utility for constructing an instance of the passed HMaster class.
3043   * @param masterClass
3044   * @return HMaster instance.
3045   */
3046  public static HMaster constructMaster(Class<? extends HMaster> masterClass,
3047      final Configuration conf)  {
3048    try {
3049      Constructor<? extends HMaster> c = masterClass.getConstructor(Configuration.class);
3050      return c.newInstance(conf);
3051    } catch(Exception e) {
3052      Throwable error = e;
3053      if (e instanceof InvocationTargetException &&
3054          ((InvocationTargetException)e).getTargetException() != null) {
3055        error = ((InvocationTargetException)e).getTargetException();
3056      }
3057      throw new RuntimeException("Failed construction of Master: " + masterClass.toString() + ". "
3058        , error);
3059    }
3060  }
3061
3062  /**
3063   * @see org.apache.hadoop.hbase.master.HMasterCommandLine
3064   */
3065  public static void main(String [] args) {
3066    LOG.info("STARTING service " + HMaster.class.getSimpleName());
3067    VersionInfo.logVersion();
3068    new HMasterCommandLine(HMaster.class).doMain(args);
3069  }
3070
3071  public HFileCleaner getHFileCleaner() {
3072    return this.hfileCleaner;
3073  }
3074
3075  public LogCleaner getLogCleaner() {
3076    return this.logCleaner;
3077  }
3078
3079  /**
3080   * @return the underlying snapshot manager
3081   */
3082  @Override
3083  public SnapshotManager getSnapshotManager() {
3084    return this.snapshotManager;
3085  }
3086
3087  /**
3088   * @return the underlying MasterProcedureManagerHost
3089   */
3090  @Override
3091  public MasterProcedureManagerHost getMasterProcedureManagerHost() {
3092    return mpmHost;
3093  }
3094
3095  @Override
3096  public ClusterSchema getClusterSchema() {
3097    return this.clusterSchemaService;
3098  }
3099
3100  /**
3101   * Create a new Namespace.
3102   * @param namespaceDescriptor descriptor for new Namespace
3103   * @param nonceGroup Identifier for the source of the request, a client or process.
3104   * @param nonce A unique identifier for this operation from the client or process identified by
3105   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3106   * @return procedure id
3107   */
3108  long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
3109      final long nonce) throws IOException {
3110    checkInitialized();
3111
3112    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
3113
3114    return MasterProcedureUtil.submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this,
3115          nonceGroup, nonce) {
3116      @Override
3117      protected void run() throws IOException {
3118        getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor);
3119        // We need to wait for the procedure to potentially fail due to "prepare" sanity
3120        // checks. This will block only the beginning of the procedure. See HBASE-19953.
3121        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3122        LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
3123        // Execute the operation synchronously - wait for the operation to complete before
3124        // continuing.
3125        setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey(), latch));
3126        latch.await();
3127        getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);
3128      }
3129
3130      @Override
3131      protected String getDescription() {
3132        return "CreateNamespaceProcedure";
3133      }
3134    });
3135  }
3136
3137  /**
3138   * Modify an existing Namespace.
3139   * @param nonceGroup Identifier for the source of the request, a client or process.
3140   * @param nonce A unique identifier for this operation from the client or process identified by
3141   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3142   * @return procedure id
3143   */
3144  long modifyNamespace(final NamespaceDescriptor newNsDescriptor, final long nonceGroup,
3145      final long nonce) throws IOException {
3146    checkInitialized();
3147
3148    TableName.isLegalNamespaceName(Bytes.toBytes(newNsDescriptor.getName()));
3149
3150    return MasterProcedureUtil.submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this,
3151          nonceGroup, nonce) {
3152      @Override
3153      protected void run() throws IOException {
3154        NamespaceDescriptor oldNsDescriptor = getNamespace(newNsDescriptor.getName());
3155        getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor, newNsDescriptor);
3156        // We need to wait for the procedure to potentially fail due to "prepare" sanity
3157        // checks. This will block only the beginning of the procedure. See HBASE-19953.
3158        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3159        LOG.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor);
3160        // Execute the operation synchronously - wait for the operation to complete before
3161        // continuing.
3162        setProcId(getClusterSchema().modifyNamespace(newNsDescriptor, getNonceKey(), latch));
3163        latch.await();
3164        getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor,
3165          newNsDescriptor);
3166      }
3167
3168      @Override
3169      protected String getDescription() {
3170        return "ModifyNamespaceProcedure";
3171      }
3172    });
3173  }
3174
3175  /**
3176   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
3177   * @param nonceGroup Identifier for the source of the request, a client or process.
3178   * @param nonce A unique identifier for this operation from the client or process identified by
3179   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3180   * @return procedure id
3181   */
3182  long deleteNamespace(final String name, final long nonceGroup, final long nonce)
3183      throws IOException {
3184    checkInitialized();
3185
3186    return MasterProcedureUtil.submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this,
3187          nonceGroup, nonce) {
3188      @Override
3189      protected void run() throws IOException {
3190        getMaster().getMasterCoprocessorHost().preDeleteNamespace(name);
3191        LOG.info(getClientIdAuditPrefix() + " delete " + name);
3192        // Execute the operation synchronously - wait for the operation to complete before
3193        // continuing.
3194        //
3195        // We need to wait for the procedure to potentially fail due to "prepare" sanity
3196        // checks. This will block only the beginning of the procedure. See HBASE-19953.
3197        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3198        setProcId(submitProcedure(
3199              new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name, latch)));
3200        latch.await();
3201        // Will not be invoked in the face of Exception thrown by the Procedure's execution
3202        getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);
3203      }
3204
3205      @Override
3206      protected String getDescription() {
3207        return "DeleteNamespaceProcedure";
3208      }
3209    });
3210  }
3211
3212  /**
3213   * Get a Namespace
3214   * @param name Name of the Namespace
3215   * @return Namespace descriptor for <code>name</code>
3216   */
3217  NamespaceDescriptor getNamespace(String name) throws IOException {
3218    checkInitialized();
3219    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
3220    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
3221    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
3222    return nsd;
3223  }
3224
3225  /**
3226   * Get all Namespaces
3227   * @return All Namespace descriptors
3228   */
3229  List<NamespaceDescriptor> getNamespaces() throws IOException {
3230    checkInitialized();
3231    final List<NamespaceDescriptor> nsds = new ArrayList<>();
3232    if (cpHost != null) {
3233      cpHost.preListNamespaceDescriptors(nsds);
3234    }
3235    nsds.addAll(this.clusterSchemaService.getNamespaces());
3236    if (this.cpHost != null) {
3237      this.cpHost.postListNamespaceDescriptors(nsds);
3238    }
3239    return nsds;
3240  }
3241
3242  /**
3243   * List namespace names
3244   * @return All namespace names
3245   */
3246  public List<String> listNamespaces() throws IOException {
3247    checkInitialized();
3248    List<String> namespaces = new ArrayList<>();
3249    if (cpHost != null) {
3250      cpHost.preListNamespaces(namespaces);
3251    }
3252    for (NamespaceDescriptor namespace : clusterSchemaService.getNamespaces()) {
3253      namespaces.add(namespace.getName());
3254    }
3255    if (cpHost != null) {
3256      cpHost.postListNamespaces(namespaces);
3257    }
3258    return namespaces;
3259  }
3260
3261  @Override
3262  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
3263    checkInitialized();
3264    return listTableNames(name, null, true);
3265  }
3266
3267  @Override
3268  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
3269    checkInitialized();
3270    return listTableDescriptors(name, null, null, true);
3271  }
3272
3273  @Override
3274  public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
3275      throws IOException {
3276    if (cpHost != null) {
3277      cpHost.preAbortProcedure(this.procedureExecutor, procId);
3278    }
3279
3280    final boolean result = this.procedureExecutor.abort(procId, mayInterruptIfRunning);
3281
3282    if (cpHost != null) {
3283      cpHost.postAbortProcedure();
3284    }
3285
3286    return result;
3287  }
3288
3289  @Override
3290  public List<Procedure<?>> getProcedures() throws IOException {
3291    if (cpHost != null) {
3292      cpHost.preGetProcedures();
3293    }
3294
3295    @SuppressWarnings({ "unchecked", "rawtypes" })
3296    List<Procedure<?>> procList = (List) this.procedureExecutor.getProcedures();
3297
3298    if (cpHost != null) {
3299      cpHost.postGetProcedures(procList);
3300    }
3301
3302    return procList;
3303  }
3304
3305  @Override
3306  public List<LockedResource> getLocks() throws IOException {
3307    if (cpHost != null) {
3308      cpHost.preGetLocks();
3309    }
3310
3311    MasterProcedureScheduler procedureScheduler =
3312      procedureExecutor.getEnvironment().getProcedureScheduler();
3313
3314    final List<LockedResource> lockedResources = procedureScheduler.getLocks();
3315
3316    if (cpHost != null) {
3317      cpHost.postGetLocks(lockedResources);
3318    }
3319
3320    return lockedResources;
3321  }
3322
3323  /**
3324   * Returns the list of table descriptors that match the specified request
3325   * @param namespace the namespace to query, or null if querying for all
3326   * @param regex The regular expression to match against, or null if querying for all
3327   * @param tableNameList the list of table names, or null if querying for all
3328   * @param includeSysTables False to match only against userspace tables
3329   * @return the list of table descriptors
3330   */
3331  public List<TableDescriptor> listTableDescriptors(final String namespace, final String regex,
3332      final List<TableName> tableNameList, final boolean includeSysTables)
3333  throws IOException {
3334    List<TableDescriptor> htds = new ArrayList<>();
3335    if (cpHost != null) {
3336      cpHost.preGetTableDescriptors(tableNameList, htds, regex);
3337    }
3338    htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);
3339    if (cpHost != null) {
3340      cpHost.postGetTableDescriptors(tableNameList, htds, regex);
3341    }
3342    return htds;
3343  }
3344
3345  /**
3346   * Returns the list of table names that match the specified request
3347   * @param regex The regular expression to match against, or null if querying for all
3348   * @param namespace the namespace to query, or null if querying for all
3349   * @param includeSysTables False to match only against userspace tables
3350   * @return the list of table names
3351   */
3352  public List<TableName> listTableNames(final String namespace, final String regex,
3353      final boolean includeSysTables) throws IOException {
3354    List<TableDescriptor> htds = new ArrayList<>();
3355    if (cpHost != null) {
3356      cpHost.preGetTableNames(htds, regex);
3357    }
3358    htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
3359    if (cpHost != null) {
3360      cpHost.postGetTableNames(htds, regex);
3361    }
3362    List<TableName> result = new ArrayList<>(htds.size());
3363    for (TableDescriptor htd: htds) result.add(htd.getTableName());
3364    return result;
3365  }
3366
3367  /**
3368   * @return list of table table descriptors after filtering by regex and whether to include system
3369   *    tables, etc.
3370   * @throws IOException
3371   */
3372  private List<TableDescriptor> getTableDescriptors(final List<TableDescriptor> htds,
3373      final String namespace, final String regex, final List<TableName> tableNameList,
3374      final boolean includeSysTables)
3375  throws IOException {
3376    if (tableNameList == null || tableNameList.isEmpty()) {
3377      // request for all TableDescriptors
3378      Collection<TableDescriptor> allHtds;
3379      if (namespace != null && namespace.length() > 0) {
3380        // Do a check on the namespace existence. Will fail if does not exist.
3381        this.clusterSchemaService.getNamespace(namespace);
3382        allHtds = tableDescriptors.getByNamespace(namespace).values();
3383      } else {
3384        allHtds = tableDescriptors.getAll().values();
3385      }
3386      for (TableDescriptor desc: allHtds) {
3387        if (tableStateManager.isTablePresent(desc.getTableName())
3388            && (includeSysTables || !desc.getTableName().isSystemTable())) {
3389          htds.add(desc);
3390        }
3391      }
3392    } else {
3393      for (TableName s: tableNameList) {
3394        if (tableStateManager.isTablePresent(s)) {
3395          TableDescriptor desc = tableDescriptors.get(s);
3396          if (desc != null) {
3397            htds.add(desc);
3398          }
3399        }
3400      }
3401    }
3402
3403    // Retains only those matched by regular expression.
3404    if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
3405    return htds;
3406  }
3407
3408  /**
3409   * Removes the table descriptors that don't match the pattern.
3410   * @param descriptors list of table descriptors to filter
3411   * @param pattern the regex to use
3412   */
3413  private static void filterTablesByRegex(final Collection<TableDescriptor> descriptors,
3414      final Pattern pattern) {
3415    final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
3416    Iterator<TableDescriptor> itr = descriptors.iterator();
3417    while (itr.hasNext()) {
3418      TableDescriptor htd = itr.next();
3419      String tableName = htd.getTableName().getNameAsString();
3420      boolean matched = pattern.matcher(tableName).matches();
3421      if (!matched && htd.getTableName().getNamespaceAsString().equals(defaultNS)) {
3422        matched = pattern.matcher(defaultNS + TableName.NAMESPACE_DELIM + tableName).matches();
3423      }
3424      if (!matched) {
3425        itr.remove();
3426      }
3427    }
3428  }
3429
3430  @Override
3431  public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
3432    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3433        .getLastMajorCompactionTimestamp(table);
3434  }
3435
3436  @Override
3437  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
3438    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3439        .getLastMajorCompactionTimestamp(regionName);
3440  }
3441
3442  /**
3443   * Gets the mob file compaction state for a specific table.
3444   * Whether all the mob files are selected is known during the compaction execution, but
3445   * the statistic is done just before compaction starts, it is hard to know the compaction
3446   * type at that time, so the rough statistics are chosen for the mob file compaction. Only two
3447   * compaction states are available, CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
3448   * @param tableName The current table name.
3449   * @return If a given table is in mob file compaction now.
3450   */
3451  public CompactionState getMobCompactionState(TableName tableName) {
3452    AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3453    if (compactionsCount != null && compactionsCount.get() != 0) {
3454      return CompactionState.MAJOR_AND_MINOR;
3455    }
3456    return CompactionState.NONE;
3457  }
3458
3459  public void reportMobCompactionStart(TableName tableName) throws IOException {
3460    IdLock.Entry lockEntry = null;
3461    try {
3462      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3463      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3464      if (compactionsCount == null) {
3465        compactionsCount = new AtomicInteger(0);
3466        mobCompactionStates.put(tableName, compactionsCount);
3467      }
3468      compactionsCount.incrementAndGet();
3469    } finally {
3470      if (lockEntry != null) {
3471        mobCompactionLock.releaseLockEntry(lockEntry);
3472      }
3473    }
3474  }
3475
3476  public void reportMobCompactionEnd(TableName tableName) throws IOException {
3477    IdLock.Entry lockEntry = null;
3478    try {
3479      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3480      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3481      if (compactionsCount != null) {
3482        int count = compactionsCount.decrementAndGet();
3483        // remove the entry if the count is 0.
3484        if (count == 0) {
3485          mobCompactionStates.remove(tableName);
3486        }
3487      }
3488    } finally {
3489      if (lockEntry != null) {
3490        mobCompactionLock.releaseLockEntry(lockEntry);
3491      }
3492    }
3493  }
3494
3495  /**
3496   * Requests mob compaction.
3497   * @param tableName The table the compact.
3498   * @param columns The compacted columns.
3499   * @param allFiles Whether add all mob files into the compaction.
3500   */
3501  public void requestMobCompaction(TableName tableName,
3502                                   List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
3503    mobCompactThread.requestMobCompaction(conf, getFileSystem(), tableName, columns, allFiles);
3504  }
3505
3506  /**
3507   * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
3508   * false is returned.
3509   *
3510   * @return The state of the load balancer, or false if the load balancer isn't defined.
3511   */
3512  public boolean isBalancerOn() {
3513    return !isInMaintenanceMode()
3514        && loadBalancerTracker != null
3515        && loadBalancerTracker.isBalancerOn();
3516  }
3517
3518  /**
3519   * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized,
3520   * false is returned.
3521   */
3522  public boolean isNormalizerOn() {
3523    return !isInMaintenanceMode()
3524        && regionNormalizerTracker != null
3525        && regionNormalizerTracker.isNormalizerOn();
3526  }
3527
3528  /**
3529   * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized,
3530   * false is returned. If switchType is illegal, false will return.
3531   * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
3532   * @return The state of the switch
3533   */
3534  @Override
3535  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
3536    return !isInMaintenanceMode()
3537        && splitOrMergeTracker != null
3538        && splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
3539  }
3540
3541  /**
3542   * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3543   *
3544   * @return The name of the {@link LoadBalancer} in use.
3545   */
3546  public String getLoadBalancerClassName() {
3547    return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, LoadBalancerFactory
3548        .getDefaultLoadBalancerClass().getName());
3549  }
3550
3551  /**
3552   * @return RegionNormalizerTracker instance
3553   */
3554  public RegionNormalizerTracker getRegionNormalizerTracker() {
3555    return regionNormalizerTracker;
3556  }
3557
3558  public SplitOrMergeTracker getSplitOrMergeTracker() {
3559    return splitOrMergeTracker;
3560  }
3561
3562  @Override
3563  public LoadBalancer getLoadBalancer() {
3564    return balancer;
3565  }
3566
3567  @Override
3568  public FavoredNodesManager getFavoredNodesManager() {
3569    return favoredNodesManager;
3570  }
3571
3572  private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException {
3573    long procId = procedureExecutor.submitProcedure(procedure);
3574    procedure.getLatch().await();
3575    return procId;
3576  }
3577
3578  @Override
3579  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
3580      throws ReplicationException, IOException {
3581    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" +
3582      peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
3583    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
3584  }
3585
3586  @Override
3587  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
3588    LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
3589    return executePeerProcedure(new RemovePeerProcedure(peerId));
3590  }
3591
3592  @Override
3593  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
3594    LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
3595    return executePeerProcedure(new EnablePeerProcedure(peerId));
3596  }
3597
3598  @Override
3599  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
3600    LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
3601    return executePeerProcedure(new DisablePeerProcedure(peerId));
3602  }
3603
3604  @Override
3605  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
3606      throws ReplicationException, IOException {
3607    if (cpHost != null) {
3608      cpHost.preGetReplicationPeerConfig(peerId);
3609    }
3610    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
3611    ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId)
3612        .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId));
3613    if (cpHost != null) {
3614      cpHost.postGetReplicationPeerConfig(peerId);
3615    }
3616    return peerConfig;
3617  }
3618
3619  @Override
3620  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
3621      throws ReplicationException, IOException {
3622    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId +
3623      ", config=" + peerConfig);
3624    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
3625  }
3626
3627  @Override
3628  public List<ReplicationPeerDescription> listReplicationPeers(String regex)
3629      throws ReplicationException, IOException {
3630    if (cpHost != null) {
3631      cpHost.preListReplicationPeers(regex);
3632    }
3633    LOG.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex);
3634    Pattern pattern = regex == null ? null : Pattern.compile(regex);
3635    List<ReplicationPeerDescription> peers =
3636      this.replicationPeerManager.listPeers(pattern);
3637    if (cpHost != null) {
3638      cpHost.postListReplicationPeers(regex);
3639    }
3640    return peers;
3641  }
3642
3643  /**
3644   * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
3645   * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
3646   * @param servers Region servers to decommission.
3647   */
3648  public void decommissionRegionServers(final List<ServerName> servers, final boolean offload)
3649      throws HBaseIOException {
3650    List<ServerName> serversAdded = new ArrayList<>(servers.size());
3651    // Place the decommission marker first.
3652    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
3653    for (ServerName server : servers) {
3654      try {
3655        String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
3656        ZKUtil.createAndFailSilent(getZooKeeper(), node);
3657      } catch (KeeperException ke) {
3658        throw new HBaseIOException(
3659          this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);
3660      }
3661      if (this.serverManager.addServerToDrainList(server)) {
3662        serversAdded.add(server);
3663      };
3664    }
3665    // Move the regions off the decommissioned servers.
3666    if (offload) {
3667      final List<ServerName> destServers = this.serverManager.createDestinationServersList();
3668      for (ServerName server : serversAdded) {
3669        final List<RegionInfo> regionsOnServer = this.assignmentManager.getRegionsOnServer(server);
3670        for (RegionInfo hri : regionsOnServer) {
3671          ServerName dest = balancer.randomAssignment(hri, destServers);
3672          if (dest == null) {
3673            throw new HBaseIOException("Unable to determine a plan to move " + hri);
3674          }
3675          RegionPlan rp = new RegionPlan(hri, server, dest);
3676          this.assignmentManager.moveAsync(rp);
3677        }
3678      }
3679    }
3680  }
3681
3682  /**
3683   * List region servers marked as decommissioned (previously called 'draining') to not get regions
3684   * assigned to them.
3685   * @return List of decommissioned servers.
3686   */
3687  public List<ServerName> listDecommissionedRegionServers() {
3688    return this.serverManager.getDrainingServersList();
3689  }
3690
3691  /**
3692   * Remove decommission marker (previously called 'draining') from a region server to allow regions
3693   * assignments. Load regions onto the server asynchronously if a list of regions is given
3694   * @param server Region server to remove decommission marker from.
3695   */
3696  public void recommissionRegionServer(final ServerName server,
3697      final List<byte[]> encodedRegionNames) throws IOException {
3698    // Remove the server from decommissioned (draining) server list.
3699    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
3700    String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
3701    try {
3702      ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);
3703    } catch (KeeperException ke) {
3704      throw new HBaseIOException(
3705        this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);
3706    }
3707    this.serverManager.removeServerFromDrainList(server);
3708
3709    // Load the regions onto the server if we are given a list of regions.
3710    if (encodedRegionNames == null || encodedRegionNames.isEmpty()) {
3711      return;
3712    }
3713    if (!this.serverManager.isServerOnline(server)) {
3714      return;
3715    }
3716    for (byte[] encodedRegionName : encodedRegionNames) {
3717      RegionState regionState =
3718        assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
3719      if (regionState == null) {
3720        LOG.warn("Unknown region " + Bytes.toStringBinary(encodedRegionName));
3721        continue;
3722      }
3723      RegionInfo hri = regionState.getRegion();
3724      if (server.equals(regionState.getServerName())) {
3725        LOG.info("Skipping move of region " + hri.getRegionNameAsString() +
3726          " because region already assigned to the same server " + server + ".");
3727        continue;
3728      }
3729      RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), server);
3730      this.assignmentManager.moveAsync(rp);
3731    }
3732  }
3733
3734  @Override
3735  public LockManager getLockManager() {
3736    return lockManager;
3737  }
3738
3739  public QuotaObserverChore getQuotaObserverChore() {
3740    return this.quotaObserverChore;
3741  }
3742
3743  public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
3744    return this.spaceQuotaSnapshotNotifier;
3745  }
3746
3747  @SuppressWarnings("unchecked")
3748  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
3749    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
3750    if (procedure == null) {
3751      return null;
3752    }
3753    assert procedure instanceof RemoteProcedure;
3754    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
3755  }
3756
3757  public void remoteProcedureCompleted(long procId) {
3758    LOG.debug("Remote procedure done, pid={}", procId);
3759    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
3760    if (procedure != null) {
3761      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
3762    }
3763  }
3764
3765  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
3766    LOG.debug("Remote procedure failed, pid={}", procId, error);
3767    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
3768    if (procedure != null) {
3769      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
3770    }
3771  }
3772
3773  /**
3774   * Reopen regions provided in the argument
3775   *
3776   * @param tableName The current table name
3777   * @param regionNames The region names of the regions to reopen
3778   * @param nonceGroup Identifier for the source of the request, a client or process
3779   * @param nonce A unique identifier for this operation from the client or process identified by
3780   *   <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
3781   * @return procedure Id
3782   * @throws IOException if reopening region fails while running procedure
3783   */
3784  long reopenRegions(final TableName tableName, final List<byte[]> regionNames,
3785      final long nonceGroup, final long nonce)
3786      throws IOException {
3787
3788    return MasterProcedureUtil
3789      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3790
3791        @Override
3792        protected void run() throws IOException {
3793          submitProcedure(new ReopenTableRegionsProcedure(tableName, regionNames));
3794        }
3795
3796        @Override
3797        protected String getDescription() {
3798          return "ReopenTableRegionsProcedure";
3799        }
3800
3801      });
3802
3803  }
3804
3805  @Override
3806  public ReplicationPeerManager getReplicationPeerManager() {
3807    return replicationPeerManager;
3808  }
3809
3810  public HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>>
3811      getReplicationLoad(ServerName[] serverNames) {
3812    List<ReplicationPeerDescription> peerList = this.getReplicationPeerManager().listPeers(null);
3813    if (peerList == null) {
3814      return null;
3815    }
3816    HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>> replicationLoadSourceMap =
3817        new HashMap<>(peerList.size());
3818    peerList.stream()
3819        .forEach(peer -> replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList<>()));
3820    for (ServerName serverName : serverNames) {
3821      List<ReplicationLoadSource> replicationLoadSources =
3822          getServerManager().getLoad(serverName).getReplicationLoadSourceList();
3823      for (ReplicationLoadSource replicationLoadSource : replicationLoadSources) {
3824        replicationLoadSourceMap.get(replicationLoadSource.getPeerID())
3825            .add(new Pair<>(serverName, replicationLoadSource));
3826      }
3827    }
3828    for (List<Pair<ServerName, ReplicationLoadSource>> loads : replicationLoadSourceMap.values()) {
3829      if (loads.size() > 0) {
3830        loads.sort(Comparator.comparingLong(load -> (-1) * load.getSecond().getReplicationLag()));
3831      }
3832    }
3833    return replicationLoadSourceMap;
3834  }
3835
3836  /**
3837   * This method modifies the master's configuration in order to inject replication-related features
3838   */
3839  @VisibleForTesting
3840  public static void decorateMasterConfiguration(Configuration conf) {
3841    String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
3842    String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
3843    if (!plugins.contains(cleanerClass)) {
3844      conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
3845    }
3846    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
3847      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
3848      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
3849      if (!plugins.contains(cleanerClass)) {
3850        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
3851      }
3852    }
3853  }
3854
3855  @Override
3856  public Map<String, ReplicationStatus> getWalGroupsReplicationStatus() {
3857    if (!this.isOnline() || !LoadBalancer.isMasterCanHostUserRegions(conf)) {
3858      return new HashMap<>();
3859    }
3860    return super.getWalGroupsReplicationStatus();
3861  }
3862
3863  public HbckChore getHbckChore() {
3864    return this.hbckChore;
3865  }
3866
3867  public Optional<ServerName> getActiveMaster() {
3868    return activeMasterManager.getActiveMasterServerName();
3869  }
3870
3871  public void runReplicationBarrierCleaner() {
3872    ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;
3873    if (rbc != null) {
3874      rbc.chore();
3875    }
3876  }
3877
3878  public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() {
3879    return this.snapshotQuotaChore;
3880  }
3881
3882  public String getClusterId() {
3883    if (activeMaster) {
3884      return super.getClusterId();
3885    }
3886    return cachedClusterId.getFromCacheOrFetch();
3887  }
3888
3889  public MetaRegionLocationCache getMetaRegionLocationCache() {
3890    return this.metaRegionLocationCache;
3891  }
3892}