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.assignment;
019
020import edu.umd.cs.findbugs.annotations.NonNull;
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.Collection;
024import java.util.Collections;
025import java.util.Comparator;
026import java.util.HashMap;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Map;
030import java.util.Set;
031import java.util.SortedSet;
032import java.util.TreeSet;
033import java.util.concurrent.CompletableFuture;
034import java.util.concurrent.Future;
035import java.util.concurrent.TimeUnit;
036import java.util.concurrent.atomic.AtomicBoolean;
037import java.util.concurrent.locks.Condition;
038import java.util.concurrent.locks.ReentrantLock;
039import java.util.function.Consumer;
040import java.util.function.Function;
041import java.util.stream.Collectors;
042import java.util.stream.Stream;
043import org.apache.hadoop.conf.Configuration;
044import org.apache.hadoop.hbase.CatalogFamilyFormat;
045import org.apache.hadoop.hbase.DoNotRetryIOException;
046import org.apache.hadoop.hbase.HBaseIOException;
047import org.apache.hadoop.hbase.HConstants;
048import org.apache.hadoop.hbase.PleaseHoldException;
049import org.apache.hadoop.hbase.ServerName;
050import org.apache.hadoop.hbase.TableName;
051import org.apache.hadoop.hbase.UnknownRegionException;
052import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
053import org.apache.hadoop.hbase.client.MasterSwitchType;
054import org.apache.hadoop.hbase.client.RegionInfo;
055import org.apache.hadoop.hbase.client.RegionInfoBuilder;
056import org.apache.hadoop.hbase.client.RegionReplicaUtil;
057import org.apache.hadoop.hbase.client.RegionStatesCount;
058import org.apache.hadoop.hbase.client.Result;
059import org.apache.hadoop.hbase.client.ResultScanner;
060import org.apache.hadoop.hbase.client.Scan;
061import org.apache.hadoop.hbase.client.TableDescriptor;
062import org.apache.hadoop.hbase.client.TableState;
063import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
064import org.apache.hadoop.hbase.favored.FavoredNodesManager;
065import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
066import org.apache.hadoop.hbase.master.LoadBalancer;
067import org.apache.hadoop.hbase.master.MasterServices;
068import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
069import org.apache.hadoop.hbase.master.RegionPlan;
070import org.apache.hadoop.hbase.master.RegionState;
071import org.apache.hadoop.hbase.master.RegionState.State;
072import org.apache.hadoop.hbase.master.ServerManager;
073import org.apache.hadoop.hbase.master.TableStateManager;
074import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
075import org.apache.hadoop.hbase.master.procedure.HBCKServerCrashProcedure;
076import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
077import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
078import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
079import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
080import org.apache.hadoop.hbase.master.procedure.TruncateRegionProcedure;
081import org.apache.hadoop.hbase.master.region.MasterRegion;
082import org.apache.hadoop.hbase.procedure2.Procedure;
083import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
084import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
085import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
086import org.apache.hadoop.hbase.procedure2.util.StringUtils;
087import org.apache.hadoop.hbase.regionserver.SequenceId;
088import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
089import org.apache.hadoop.hbase.util.Bytes;
090import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
091import org.apache.hadoop.hbase.util.FutureUtils;
092import org.apache.hadoop.hbase.util.Pair;
093import org.apache.hadoop.hbase.util.Threads;
094import org.apache.hadoop.hbase.util.VersionInfo;
095import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
096import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
097import org.apache.yetus.audience.InterfaceAudience;
098import org.apache.zookeeper.KeeperException;
099import org.slf4j.Logger;
100import org.slf4j.LoggerFactory;
101
102import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
103import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
104import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
105import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
106import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
107
108/**
109 * The AssignmentManager is the coordinator for region assign/unassign operations.
110 * <ul>
111 * <li>In-memory states of regions and servers are stored in {@link RegionStates}.</li>
112 * <li>hbase:meta state updates are handled by {@link RegionStateStore}.</li>
113 * </ul>
114 * Regions are created by CreateTable, Split, Merge. Regions are deleted by DeleteTable, Split,
115 * Merge. Assigns are triggered by CreateTable, EnableTable, Split, Merge, ServerCrash. Unassigns
116 * are triggered by DisableTable, Split, Merge
117 */
118@InterfaceAudience.Private
119public class AssignmentManager {
120  private static final Logger LOG = LoggerFactory.getLogger(AssignmentManager.class);
121
122  // TODO: AMv2
123  // - handle region migration from hbase1 to hbase2.
124  // - handle sys table assignment first (e.g. acl, namespace)
125  // - handle table priorities
126  // - If ServerBusyException trying to update hbase:meta, we abort the Master
127  // See updateRegionLocation in RegionStateStore.
128  //
129  // See also
130  // https://docs.google.com/document/d/1eVKa7FHdeoJ1-9o8yZcOTAQbv0u0bblBlCCzVSIn69g/edit#heading=h.ystjyrkbtoq5
131  // for other TODOs.
132
133  public static final String BOOTSTRAP_THREAD_POOL_SIZE_CONF_KEY =
134    "hbase.assignment.bootstrap.thread.pool.size";
135
136  public static final String ASSIGN_DISPATCH_WAIT_MSEC_CONF_KEY =
137    "hbase.assignment.dispatch.wait.msec";
138  private static final int DEFAULT_ASSIGN_DISPATCH_WAIT_MSEC = 150;
139
140  public static final String ASSIGN_DISPATCH_WAITQ_MAX_CONF_KEY =
141    "hbase.assignment.dispatch.wait.queue.max.size";
142  private static final int DEFAULT_ASSIGN_DISPATCH_WAITQ_MAX = 100;
143
144  public static final String RIT_CHORE_INTERVAL_MSEC_CONF_KEY =
145    "hbase.assignment.rit.chore.interval.msec";
146  private static final int DEFAULT_RIT_CHORE_INTERVAL_MSEC = 60 * 1000;
147
148  public static final String DEAD_REGION_METRIC_CHORE_INTERVAL_MSEC_CONF_KEY =
149    "hbase.assignment.dead.region.metric.chore.interval.msec";
150  private static final int DEFAULT_DEAD_REGION_METRIC_CHORE_INTERVAL_MSEC = 120 * 1000;
151
152  public static final String ASSIGN_MAX_ATTEMPTS = "hbase.assignment.maximum.attempts";
153  private static final int DEFAULT_ASSIGN_MAX_ATTEMPTS = Integer.MAX_VALUE;
154
155  public static final String ASSIGN_RETRY_IMMEDIATELY_MAX_ATTEMPTS =
156    "hbase.assignment.retry.immediately.maximum.attempts";
157  private static final int DEFAULT_ASSIGN_RETRY_IMMEDIATELY_MAX_ATTEMPTS = 3;
158
159  /** Region in Transition metrics threshold time */
160  public static final String METRICS_RIT_STUCK_WARNING_THRESHOLD =
161    "hbase.metrics.rit.stuck.warning.threshold";
162  private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
163  public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
164
165  public static final String FORCE_REGION_RETAINMENT = "hbase.master.scp.retain.assignment.force";
166
167  public static final boolean DEFAULT_FORCE_REGION_RETAINMENT = false;
168
169  /** The wait time in millis before checking again if the region's previous RS is back online */
170  public static final String FORCE_REGION_RETAINMENT_WAIT_INTERVAL =
171    "hbase.master.scp.retain.assignment.force.wait-interval";
172
173  public static final long DEFAULT_FORCE_REGION_RETAINMENT_WAIT_INTERVAL = 50;
174
175  /**
176   * The number of times to check if the region's previous RS is back online, before giving up and
177   * proceeding with assignment on a new RS
178   */
179  public static final String FORCE_REGION_RETAINMENT_RETRIES =
180    "hbase.master.scp.retain.assignment.force.retries";
181
182  public static final int DEFAULT_FORCE_REGION_RETAINMENT_RETRIES = 600;
183
184  private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
185  private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
186
187  private final MetricsAssignmentManager metrics;
188  private final RegionInTransitionChore ritChore;
189  private final DeadServerMetricRegionChore deadMetricChore;
190  private final MasterServices master;
191
192  private final AtomicBoolean running = new AtomicBoolean(false);
193  private final RegionStates regionStates = new RegionStates();
194  private final RegionStateStore regionStateStore;
195
196  /**
197   * When the operator uses this configuration option, any version between the current cluster
198   * version and the value of "hbase.min.version.move.system.tables" does not trigger any
199   * auto-region movement. Auto-region movement here refers to auto-migration of system table
200   * regions to newer server versions. It is assumed that the configured range of versions does not
201   * require special handling of moving system table regions to higher versioned RegionServer. This
202   * auto-migration is done by {@link #checkIfShouldMoveSystemRegionAsync()}. Example: Let's assume
203   * the cluster is on version 1.4.0 and we have set "hbase.min.version.move.system.tables" as
204   * "2.0.0". Now if we upgrade one RegionServer on 1.4.0 cluster to 1.6.0 (< 2.0.0), then
205   * AssignmentManager will not move hbase:meta, hbase:namespace and other system table regions to
206   * newly brought up RegionServer 1.6.0 as part of auto-migration. However, if we upgrade one
207   * RegionServer on 1.4.0 cluster to 2.2.0 (> 2.0.0), then AssignmentManager will move all system
208   * table regions to newly brought up RegionServer 2.2.0 as part of auto-migration done by
209   * {@link #checkIfShouldMoveSystemRegionAsync()}. "hbase.min.version.move.system.tables" is
210   * introduced as part of HBASE-22923.
211   */
212  private final String minVersionToMoveSysTables;
213
214  private static final String MIN_VERSION_MOVE_SYS_TABLES_CONFIG =
215    "hbase.min.version.move.system.tables";
216  private static final String DEFAULT_MIN_VERSION_MOVE_SYS_TABLES_CONFIG = "";
217
218  private final Map<ServerName, Set<byte[]>> rsReports = new HashMap<>();
219
220  private final boolean shouldAssignRegionsWithFavoredNodes;
221  private final int assignDispatchWaitQueueMaxSize;
222  private final int assignDispatchWaitMillis;
223  private final int assignMaxAttempts;
224  private final int assignRetryImmediatelyMaxAttempts;
225
226  private final MasterRegion masterRegion;
227
228  private final Object checkIfShouldMoveSystemRegionLock = new Object();
229
230  private Thread assignThread;
231
232  private final boolean forceRegionRetainment;
233
234  private final long forceRegionRetainmentWaitInterval;
235
236  private final int forceRegionRetainmentRetries;
237
238  private final RegionInTransitionTracker regionInTransitionTracker =
239    new RegionInTransitionTracker();
240
241  public AssignmentManager(MasterServices master, MasterRegion masterRegion) {
242    this(master, masterRegion, new RegionStateStore(master, masterRegion));
243  }
244
245  AssignmentManager(MasterServices master, MasterRegion masterRegion, RegionStateStore stateStore) {
246    this.master = master;
247    this.regionStateStore = stateStore;
248    this.metrics = new MetricsAssignmentManager();
249    this.masterRegion = masterRegion;
250
251    final Configuration conf = master.getConfiguration();
252
253    // Only read favored nodes if using the favored nodes load balancer.
254    this.shouldAssignRegionsWithFavoredNodes = FavoredStochasticBalancer.class
255      .isAssignableFrom(conf.getClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class));
256
257    this.assignDispatchWaitMillis =
258      conf.getInt(ASSIGN_DISPATCH_WAIT_MSEC_CONF_KEY, DEFAULT_ASSIGN_DISPATCH_WAIT_MSEC);
259    this.assignDispatchWaitQueueMaxSize =
260      conf.getInt(ASSIGN_DISPATCH_WAITQ_MAX_CONF_KEY, DEFAULT_ASSIGN_DISPATCH_WAITQ_MAX);
261
262    this.assignMaxAttempts =
263      Math.max(1, conf.getInt(ASSIGN_MAX_ATTEMPTS, DEFAULT_ASSIGN_MAX_ATTEMPTS));
264    this.assignRetryImmediatelyMaxAttempts = conf.getInt(ASSIGN_RETRY_IMMEDIATELY_MAX_ATTEMPTS,
265      DEFAULT_ASSIGN_RETRY_IMMEDIATELY_MAX_ATTEMPTS);
266
267    int ritChoreInterval =
268      conf.getInt(RIT_CHORE_INTERVAL_MSEC_CONF_KEY, DEFAULT_RIT_CHORE_INTERVAL_MSEC);
269    this.ritChore = new RegionInTransitionChore(ritChoreInterval);
270
271    int deadRegionChoreInterval = conf.getInt(DEAD_REGION_METRIC_CHORE_INTERVAL_MSEC_CONF_KEY,
272      DEFAULT_DEAD_REGION_METRIC_CHORE_INTERVAL_MSEC);
273    if (deadRegionChoreInterval > 0) {
274      this.deadMetricChore = new DeadServerMetricRegionChore(deadRegionChoreInterval);
275    } else {
276      this.deadMetricChore = null;
277    }
278    minVersionToMoveSysTables =
279      conf.get(MIN_VERSION_MOVE_SYS_TABLES_CONFIG, DEFAULT_MIN_VERSION_MOVE_SYS_TABLES_CONFIG);
280
281    forceRegionRetainment =
282      conf.getBoolean(FORCE_REGION_RETAINMENT, DEFAULT_FORCE_REGION_RETAINMENT);
283    forceRegionRetainmentWaitInterval = conf.getLong(FORCE_REGION_RETAINMENT_WAIT_INTERVAL,
284      DEFAULT_FORCE_REGION_RETAINMENT_WAIT_INTERVAL);
285    forceRegionRetainmentRetries =
286      conf.getInt(FORCE_REGION_RETAINMENT_RETRIES, DEFAULT_FORCE_REGION_RETAINMENT_RETRIES);
287  }
288
289  private void mirrorMetaLocations() throws IOException, KeeperException {
290    // For compatibility, mirror the meta region state to zookeeper
291    // And we still need to use zookeeper to publish the meta region locations to region
292    // server, so they can serve as ClientMetaService
293    ZKWatcher zk = master.getZooKeeper();
294    if (zk == null || !zk.getRecoverableZooKeeper().getState().isAlive()) {
295      // this is possible in tests, we do not provide a zk watcher or the zk watcher has been closed
296      return;
297    }
298    Collection<RegionStateNode> metaStates = regionStates.getRegionStateNodes();
299    for (RegionStateNode metaState : metaStates) {
300      MetaTableLocator.setMetaLocation(zk, metaState.getRegionLocation(),
301        metaState.getRegionInfo().getReplicaId(), metaState.getState());
302    }
303    int replicaCount = metaStates.size();
304    // remove extra mirror locations
305    for (String znode : zk.getMetaReplicaNodes()) {
306      int replicaId = zk.getZNodePaths().getMetaReplicaIdFromZNode(znode);
307      if (replicaId >= replicaCount) {
308        MetaTableLocator.deleteMetaLocation(zk, replicaId);
309      }
310    }
311  }
312
313  public void start() throws IOException, KeeperException {
314    if (!running.compareAndSet(false, true)) {
315      return;
316    }
317
318    LOG.trace("Starting assignment manager");
319
320    // Start the Assignment Thread
321    startAssignmentThread();
322    // load meta region states.
323    // here we are still in the early steps of active master startup. There is only one thread(us)
324    // can access AssignmentManager and create region node, so here we do not need to lock the
325    // region node.
326    try (ResultScanner scanner =
327      masterRegion.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
328      for (;;) {
329        Result result = scanner.next();
330        if (result == null) {
331          break;
332        }
333        RegionStateStore
334          .visitMetaEntry((r, regionInfo, state, regionLocation, lastHost, openSeqNum) -> {
335            RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
336            regionNode.setState(state);
337            regionNode.setLastHost(lastHost);
338            regionNode.setRegionLocation(regionLocation);
339            regionNode.setOpenSeqNum(openSeqNum);
340            regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
341
342            if (regionNode.getProcedure() != null) {
343              regionNode.getProcedure().stateLoaded(this, regionNode);
344            }
345            if (regionLocation != null) {
346              // TODO: this could lead to some orphan server state nodes, as it is possible that the
347              // region server is already dead and its SCP has already finished but we have
348              // persisted an opening state on this region server. Finally the TRSP will assign the
349              // region to another region server, so it will not cause critical problems, just waste
350              // some memory as no one will try to cleanup these orphan server state nodes.
351              regionStates.createServer(regionLocation);
352              regionStates.addRegionToServer(regionNode);
353            }
354            if (RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) {
355              setMetaAssigned(regionInfo, state == State.OPEN);
356            }
357            LOG.debug("Loaded hbase:meta {}", regionNode);
358          }, result);
359      }
360    }
361    mirrorMetaLocations();
362  }
363
364  /**
365   * Create RegionStateNode based on the TRSP list, and attach the TRSP to the RegionStateNode.
366   * <p>
367   * This is used to restore the RIT region list, so we do not need to restore it in the loadingMeta
368   * method below. And it is also very important as now before submitting a TRSP, we need to attach
369   * it to the RegionStateNode, which acts like a guard, so we need to restore this information at
370   * the very beginning, before we start processing any procedures.
371   */
372  public void setupRIT(List<TransitRegionStateProcedure> procs) {
373    procs.forEach(proc -> {
374      RegionInfo regionInfo = proc.getRegion();
375      RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
376      TransitRegionStateProcedure existingProc = regionNode.getProcedure();
377      if (existingProc != null) {
378        // This is possible, as we will detach the procedure from the RSN before we
379        // actually finish the procedure. This is because that, we will detach the TRSP from the RSN
380        // during execution, at that time, the procedure has not been marked as done in the pv2
381        // framework yet, so it is possible that we schedule a new TRSP immediately and when
382        // arriving here, we will find out that there are multiple TRSPs for the region. But we can
383        // make sure that, only the last one can take the charge, the previous ones should have all
384        // been finished already. So here we will compare the proc id, the greater one will win.
385        if (existingProc.getProcId() < proc.getProcId()) {
386          // the new one wins, unset and set it to the new one below
387          regionNode.unsetProcedure(existingProc);
388        } else {
389          // the old one wins, skip
390          return;
391        }
392      }
393      LOG.info("Attach {} to {}", proc, regionNode);
394      regionNode.setProcedure(proc);
395    });
396  }
397
398  public void initializationPostMetaOnline() {
399    // now that we are sure that meta is online, we can set TableStateManger in
400    // regionInTransitionTracker
401    regionInTransitionTracker.setTableStateManager(master.getTableStateManager());
402  }
403
404  public void stop() {
405    if (!running.compareAndSet(true, false)) {
406      return;
407    }
408
409    LOG.info("Stopping assignment manager");
410
411    // The AM is started before the procedure executor,
412    // but the actual work will be loaded/submitted only once we have the executor
413    final boolean hasProcExecutor = master.getMasterProcedureExecutor() != null;
414
415    // Remove the RIT chore
416    if (hasProcExecutor) {
417      master.getMasterProcedureExecutor().removeChore(this.ritChore);
418      if (this.deadMetricChore != null) {
419        master.getMasterProcedureExecutor().removeChore(this.deadMetricChore);
420      }
421    }
422
423    // Stop the Assignment Thread
424    stopAssignmentThread();
425
426    // Stop the RegionStateStore
427    regionStates.clear();
428    regionInTransitionTracker.stop();
429
430    // Update meta events (for testing)
431    if (hasProcExecutor) {
432      metaLoadEvent.suspend();
433      for (RegionInfo hri : getMetaRegionSet()) {
434        setMetaAssigned(hri, false);
435      }
436    }
437  }
438
439  public boolean isRunning() {
440    return running.get();
441  }
442
443  public Configuration getConfiguration() {
444    return master.getConfiguration();
445  }
446
447  public MetricsAssignmentManager getAssignmentManagerMetrics() {
448    return metrics;
449  }
450
451  private LoadBalancer getBalancer() {
452    return master.getLoadBalancer();
453  }
454
455  private FavoredNodesPromoter getFavoredNodePromoter() {
456    return (FavoredNodesPromoter) ((RSGroupBasedLoadBalancer) master.getLoadBalancer())
457      .getInternalBalancer();
458  }
459
460  private MasterProcedureEnv getProcedureEnvironment() {
461    return master.getMasterProcedureExecutor().getEnvironment();
462  }
463
464  private MasterProcedureScheduler getProcedureScheduler() {
465    return getProcedureEnvironment().getProcedureScheduler();
466  }
467
468  int getAssignMaxAttempts() {
469    return assignMaxAttempts;
470  }
471
472  public boolean isForceRegionRetainment() {
473    return forceRegionRetainment;
474  }
475
476  public long getForceRegionRetainmentWaitInterval() {
477    return forceRegionRetainmentWaitInterval;
478  }
479
480  public int getForceRegionRetainmentRetries() {
481    return forceRegionRetainmentRetries;
482  }
483
484  int getAssignRetryImmediatelyMaxAttempts() {
485    return assignRetryImmediatelyMaxAttempts;
486  }
487
488  public RegionStates getRegionStates() {
489    return regionStates;
490  }
491
492  /**
493   * Returns the regions hosted by the specified server.
494   * <p/>
495   * Notice that, for SCP, after we submit the SCP, no one can change the region list for the
496   * ServerStateNode so we do not need any locks here. And for other usage, this can only give you a
497   * snapshot of the current region list for this server, which means, right after you get the
498   * region list, new regions may be moved to this server or some regions may be moved out from this
499   * server, so you should not use it critically if you need strong consistency.
500   */
501  public List<RegionInfo> getRegionsOnServer(ServerName serverName) {
502    ServerStateNode serverInfo = regionStates.getServerNode(serverName);
503    if (serverInfo == null) {
504      return Collections.emptyList();
505    }
506    return serverInfo.getRegionInfoList();
507  }
508
509  private RegionInfo getRegionInfo(RegionStateNode rsn) {
510    if (rsn.isSplit() && !rsn.getRegionInfo().isSplit()) {
511      // see the comments in markRegionAsSplit on why we need to do this converting.
512      return RegionInfoBuilder.newBuilder(rsn.getRegionInfo()).setSplit(true).setOffline(true)
513        .build();
514    } else {
515      return rsn.getRegionInfo();
516    }
517  }
518
519  private Stream<RegionStateNode> getRegionStateNodes(TableName tableName,
520    boolean excludeOfflinedSplitParents) {
521    Stream<RegionStateNode> stream = regionStates.getTableRegionStateNodes(tableName).stream();
522    if (excludeOfflinedSplitParents) {
523      return stream.filter(rsn -> !rsn.isSplit());
524    } else {
525      return stream;
526    }
527  }
528
529  public List<RegionInfo> getTableRegions(TableName tableName,
530    boolean excludeOfflinedSplitParents) {
531    return getRegionStateNodes(tableName, excludeOfflinedSplitParents).map(this::getRegionInfo)
532      .collect(Collectors.toList());
533  }
534
535  public List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(TableName tableName,
536    boolean excludeOfflinedSplitParents) {
537    return getRegionStateNodes(tableName, excludeOfflinedSplitParents)
538      .map(rsn -> Pair.newPair(getRegionInfo(rsn), rsn.getRegionLocation()))
539      .collect(Collectors.toList());
540  }
541
542  public RegionStateStore getRegionStateStore() {
543    return regionStateStore;
544  }
545
546  public List<ServerName> getFavoredNodes(final RegionInfo regionInfo) {
547    return this.shouldAssignRegionsWithFavoredNodes
548      ? getFavoredNodePromoter().getFavoredNodes(regionInfo)
549      : ServerName.EMPTY_SERVER_LIST;
550  }
551
552  // ============================================================================================
553  // Table State Manager helpers
554  // ============================================================================================
555  private TableStateManager getTableStateManager() {
556    return master.getTableStateManager();
557  }
558
559  private boolean isTableEnabled(final TableName tableName) {
560    return getTableStateManager().isTableState(tableName, TableState.State.ENABLED);
561  }
562
563  private boolean isTableDisabled(final TableName tableName) {
564    return getTableStateManager().isTableState(tableName, TableState.State.DISABLED,
565      TableState.State.DISABLING);
566  }
567
568  // ============================================================================================
569  // META Helpers
570  // ============================================================================================
571  private boolean isMetaRegion(final RegionInfo regionInfo) {
572    return regionInfo.isMetaRegion();
573  }
574
575  public boolean isMetaRegion(final byte[] regionName) {
576    return getMetaRegionFromName(regionName) != null;
577  }
578
579  public RegionInfo getMetaRegionFromName(final byte[] regionName) {
580    for (RegionInfo hri : getMetaRegionSet()) {
581      if (Bytes.equals(hri.getRegionName(), regionName)) {
582        return hri;
583      }
584    }
585    return null;
586  }
587
588  public boolean isCarryingMeta(final ServerName serverName) {
589    // TODO: handle multiple meta
590    return isCarryingRegion(serverName, RegionInfoBuilder.FIRST_META_REGIONINFO);
591  }
592
593  private boolean isCarryingRegion(final ServerName serverName, final RegionInfo regionInfo) {
594    // TODO: check for state?
595    final RegionStateNode node = regionStates.getRegionStateNode(regionInfo);
596    return (node != null && serverName.equals(node.getRegionLocation()));
597  }
598
599  private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
600    // if (regionInfo.isMetaRegion()) return regionInfo;
601    // TODO: handle multiple meta. if the region provided is not meta lookup
602    // which meta the region belongs to.
603    return RegionInfoBuilder.FIRST_META_REGIONINFO;
604  }
605
606  // TODO: handle multiple meta.
607  private static final Set<RegionInfo> META_REGION_SET =
608    Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
609
610  public Set<RegionInfo> getMetaRegionSet() {
611    return META_REGION_SET;
612  }
613
614  // ============================================================================================
615  // META Event(s) helpers
616  // ============================================================================================
617  /**
618   * Notice that, this only means the meta region is available on a RS, but the AM may still be
619   * loading the region states from meta, so usually you need to check {@link #isMetaLoaded()} first
620   * before checking this method, unless you can make sure that your piece of code can only be
621   * executed after AM builds the region states.
622   * @see #isMetaLoaded()
623   */
624  public boolean isMetaAssigned() {
625    return metaAssignEvent.isReady();
626  }
627
628  public boolean isMetaRegionInTransition() {
629    return !isMetaAssigned();
630  }
631
632  /**
633   * Notice that this event does not mean the AM has already finished region state rebuilding. See
634   * the comment of {@link #isMetaAssigned()} for more details.
635   * @see #isMetaAssigned()
636   */
637  public boolean waitMetaAssigned(Procedure<?> proc, RegionInfo regionInfo) {
638    return getMetaAssignEvent(getMetaForRegion(regionInfo)).suspendIfNotReady(proc);
639  }
640
641  private void setMetaAssigned(RegionInfo metaRegionInfo, boolean assigned) {
642    assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
643    ProcedureEvent<?> metaAssignEvent = getMetaAssignEvent(metaRegionInfo);
644    if (assigned) {
645      metaAssignEvent.wake(getProcedureScheduler());
646    } else {
647      metaAssignEvent.suspend();
648    }
649  }
650
651  private ProcedureEvent<?> getMetaAssignEvent(RegionInfo metaRegionInfo) {
652    assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
653    // TODO: handle multiple meta.
654    return metaAssignEvent;
655  }
656
657  /**
658   * Wait until AM finishes the meta loading, i.e, the region states rebuilding.
659   * @see #isMetaLoaded()
660   * @see #waitMetaAssigned(Procedure, RegionInfo)
661   */
662  public boolean waitMetaLoaded(Procedure<?> proc) {
663    return metaLoadEvent.suspendIfNotReady(proc);
664  }
665
666  /**
667   * This method will be called in master initialization method after calling
668   * {@link #processOfflineRegions()}, as in processOfflineRegions we will generate assign
669   * procedures for offline regions, which may be conflict with creating table.
670   * <p/>
671   * This is a bit dirty, should be reconsidered after we decide whether to keep the
672   * {@link #processOfflineRegions()} method.
673   */
674  public void wakeMetaLoadedEvent() {
675    metaLoadEvent.wake(getProcedureScheduler());
676    assert isMetaLoaded() : "expected meta to be loaded";
677  }
678
679  /**
680   * Return whether AM finishes the meta loading, i.e, the region states rebuilding.
681   * @see #isMetaAssigned()
682   * @see #waitMetaLoaded(Procedure)
683   */
684  public boolean isMetaLoaded() {
685    return metaLoadEvent.isReady();
686  }
687
688  /**
689   * Start a new thread to check if there are region servers whose versions are higher than others.
690   * If so, move all system table regions to RS with the highest version to keep compatibility. The
691   * reason is, RS in new version may not be able to access RS in old version when there are some
692   * incompatible changes.
693   * <p>
694   * This method is called when a new RegionServer is added to cluster only.
695   * </p>
696   */
697  public void checkIfShouldMoveSystemRegionAsync() {
698    // TODO: Fix this thread. If a server is killed and a new one started, this thread thinks that
699    // it should 'move' the system tables from the old server to the new server but
700    // ServerCrashProcedure is on it; and it will take care of the assign without dataloss.
701    if (this.master.getServerManager().countOfRegionServers() <= 1) {
702      return;
703    }
704    // This thread used to run whenever there was a change in the cluster. The ZooKeeper
705    // childrenChanged notification came in before the nodeDeleted message and so this method
706    // cold run before a ServerCrashProcedure could run. That meant that this thread could see
707    // a Crashed Server before ServerCrashProcedure and it could find system regions on the
708    // crashed server and go move them before ServerCrashProcedure had a chance; could be
709    // dataloss too if WALs were not recovered.
710    new Thread(() -> {
711      try {
712        synchronized (checkIfShouldMoveSystemRegionLock) {
713          List<RegionPlan> plans = new ArrayList<>();
714          // TODO: I don't think this code does a good job if all servers in cluster have same
715          // version. It looks like it will schedule unnecessary moves.
716          for (ServerName server : getExcludedServersForSystemTable()) {
717            if (master.getServerManager().isServerDead(server)) {
718              // TODO: See HBASE-18494 and HBASE-18495. Though getExcludedServersForSystemTable()
719              // considers only online servers, the server could be queued for dead server
720              // processing. As region assignments for crashed server is handled by
721              // ServerCrashProcedure, do NOT handle them here. The goal is to handle this through
722              // regular flow of LoadBalancer as a favored node and not to have this special
723              // handling.
724              continue;
725            }
726            List<RegionInfo> regionsShouldMove = getSystemTables(server);
727            if (!regionsShouldMove.isEmpty()) {
728              for (RegionInfo regionInfo : regionsShouldMove) {
729                // null value for dest forces destination server to be selected by balancer
730                RegionPlan plan = new RegionPlan(regionInfo, server, null);
731                if (regionInfo.isMetaRegion()) {
732                  // Must move meta region first.
733                  LOG.info("Async MOVE of {} to newer Server={}", regionInfo.getEncodedName(),
734                    server);
735                  moveAsync(plan);
736                } else {
737                  plans.add(plan);
738                }
739              }
740            }
741            for (RegionPlan plan : plans) {
742              LOG.info("Async MOVE of {} to newer Server={}", plan.getRegionInfo().getEncodedName(),
743                server);
744              moveAsync(plan);
745            }
746          }
747        }
748      } catch (Throwable t) {
749        LOG.error(t.toString(), t);
750      }
751    }).start();
752  }
753
754  private List<RegionInfo> getSystemTables(ServerName serverName) {
755    ServerStateNode serverNode = regionStates.getServerNode(serverName);
756    if (serverNode == null) {
757      return Collections.emptyList();
758    }
759    return serverNode.getSystemRegionInfoList();
760  }
761
762  private void preTransitCheck(RegionStateNode regionNode, RegionState.State[] expectedStates)
763    throws HBaseIOException {
764    if (regionNode.getProcedure() != null) {
765      throw new HBaseIOException(
766        regionNode + " is currently in transition; pid=" + regionNode.getProcedure().getProcId());
767    }
768    if (!regionNode.isInState(expectedStates)) {
769      throw new DoNotRetryRegionException(UNEXPECTED_STATE_REGION + regionNode);
770    }
771    if (isTableDisabled(regionNode.getTable())) {
772      throw new DoNotRetryIOException(regionNode.getTable() + " is disabled for " + regionNode);
773    }
774  }
775
776  /**
777   * Create an assign TransitRegionStateProcedure. Makes sure of RegionState. Throws exception if
778   * not appropriate UNLESS override is set. Used by hbck2 but also by straightline
779   * {@link #assign(RegionInfo, ServerName)} and {@link #assignAsync(RegionInfo, ServerName)}.
780   * @see #createAssignProcedure(RegionStateNode, ServerName) for a version that does NO checking
781   *      used when only when no checking needed.
782   * @param override If false, check RegionState is appropriate for assign; if not throw exception.
783   */
784  private TransitRegionStateProcedure createAssignProcedure(RegionInfo regionInfo, ServerName sn,
785    boolean override, boolean force) throws IOException {
786    RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
787    regionNode.lock();
788    try {
789      if (override) {
790        if (!force) {
791          preTransitCheck(regionNode, STATES_EXPECTED_ON_ASSIGN);
792        }
793        if (regionNode.getProcedure() != null) {
794          regionNode.unsetProcedure(regionNode.getProcedure());
795        }
796      } else {
797        preTransitCheck(regionNode, STATES_EXPECTED_ON_ASSIGN);
798      }
799      assert regionNode.getProcedure() == null;
800      return regionNode.setProcedure(
801        TransitRegionStateProcedure.assign(getProcedureEnvironment(), regionInfo, sn));
802    } finally {
803      regionNode.unlock();
804    }
805  }
806
807  /**
808   * Create an assign TransitRegionStateProcedure. Does NO checking of RegionState. Presumes
809   * appriopriate state ripe for assign.
810   * @see #createAssignProcedure(RegionInfo, ServerName, boolean, boolean)
811   */
812  private TransitRegionStateProcedure createAssignProcedure(RegionStateNode regionNode,
813    ServerName targetServer) {
814    regionNode.lock();
815    try {
816      return regionNode.setProcedure(TransitRegionStateProcedure.assign(getProcedureEnvironment(),
817        regionNode.getRegionInfo(), targetServer));
818    } finally {
819      regionNode.unlock();
820    }
821  }
822
823  public long assign(RegionInfo regionInfo, ServerName sn) throws IOException {
824    TransitRegionStateProcedure proc = createAssignProcedure(regionInfo, sn, false, false);
825    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
826    return proc.getProcId();
827  }
828
829  public long assign(RegionInfo regionInfo) throws IOException {
830    return assign(regionInfo, null);
831  }
832
833  /**
834   * Submits a procedure that assigns a region to a target server without waiting for it to finish
835   * @param regionInfo the region we would like to assign
836   * @param sn         target server name
837   */
838  public Future<byte[]> assignAsync(RegionInfo regionInfo, ServerName sn) throws IOException {
839    return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(),
840      createAssignProcedure(regionInfo, sn, false, false));
841  }
842
843  /**
844   * Submits a procedure that assigns a region without waiting for it to finish
845   * @param regionInfo the region we would like to assign
846   */
847  public Future<byte[]> assignAsync(RegionInfo regionInfo) throws IOException {
848    return assignAsync(regionInfo, null);
849  }
850
851  public long unassign(RegionInfo regionInfo) throws IOException {
852    RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
853    if (regionNode == null) {
854      throw new UnknownRegionException("No RegionState found for " + regionInfo.getEncodedName());
855    }
856    TransitRegionStateProcedure proc;
857    regionNode.lock();
858    try {
859      preTransitCheck(regionNode, STATES_EXPECTED_ON_UNASSIGN_OR_MOVE);
860      proc = TransitRegionStateProcedure.unassign(getProcedureEnvironment(), regionInfo);
861      regionNode.setProcedure(proc);
862    } finally {
863      regionNode.unlock();
864    }
865    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
866    return proc.getProcId();
867  }
868
869  public TransitRegionStateProcedure createMoveRegionProcedure(RegionInfo regionInfo,
870    ServerName targetServer) throws HBaseIOException {
871    RegionStateNode regionNode = this.regionStates.getRegionStateNode(regionInfo);
872    if (regionNode == null) {
873      throw new UnknownRegionException(
874        "No RegionStateNode found for " + regionInfo.getEncodedName() + "(Closed/Deleted?)");
875    }
876    TransitRegionStateProcedure proc;
877    regionNode.lock();
878    try {
879      preTransitCheck(regionNode, STATES_EXPECTED_ON_UNASSIGN_OR_MOVE);
880      regionNode.checkOnline();
881      proc = TransitRegionStateProcedure.move(getProcedureEnvironment(), regionInfo, targetServer);
882      regionNode.setProcedure(proc);
883    } finally {
884      regionNode.unlock();
885    }
886    return proc;
887  }
888
889  public void move(RegionInfo regionInfo) throws IOException {
890    TransitRegionStateProcedure proc = createMoveRegionProcedure(regionInfo, null);
891    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
892  }
893
894  public Future<byte[]> moveAsync(RegionPlan regionPlan) throws HBaseIOException {
895    TransitRegionStateProcedure proc =
896      createMoveRegionProcedure(regionPlan.getRegionInfo(), regionPlan.getDestination());
897    return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
898  }
899
900  public Future<byte[]> balance(RegionPlan regionPlan) throws HBaseIOException {
901    ServerName current =
902      this.getRegionStates().getRegionAssignments().get(regionPlan.getRegionInfo());
903    if (current == null || !current.equals(regionPlan.getSource())) {
904      LOG.debug("Skip region plan {}, source server not match, current region location is {}",
905        regionPlan, current == null ? "(null)" : current);
906      return null;
907    }
908    return moveAsync(regionPlan);
909  }
910
911  // ============================================================================================
912  // RegionTransition procedures helpers
913  // ============================================================================================
914
915  /**
916   * Create round-robin assigns. Use on table creation to distribute out regions across cluster.
917   * @return AssignProcedures made out of the passed in <code>hris</code> and a call to the balancer
918   *         to populate the assigns with targets chosen using round-robin (default balancer
919   *         scheme). If at assign-time, the target chosen is no longer up, thats fine, the
920   *         AssignProcedure will ask the balancer for a new target, and so on.
921   */
922  public TransitRegionStateProcedure[] createRoundRobinAssignProcedures(List<RegionInfo> hris,
923    List<ServerName> serversToExclude) {
924    if (hris.isEmpty()) {
925      return new TransitRegionStateProcedure[0];
926    }
927
928    if (
929      serversToExclude != null && this.master.getServerManager().getOnlineServersList().size() == 1
930    ) {
931      LOG.debug("Only one region server found and hence going ahead with the assignment");
932      serversToExclude = null;
933    }
934    try {
935      // Ask the balancer to assign our regions. Pass the regions en masse. The balancer can do
936      // a better job if it has all the assignments in the one lump.
937      Map<ServerName, List<RegionInfo>> assignments = getBalancer().roundRobinAssignment(hris,
938        this.master.getServerManager().createDestinationServersList(serversToExclude));
939      // Return mid-method!
940      return createAssignProcedures(assignments);
941    } catch (IOException hioe) {
942      LOG.warn("Failed roundRobinAssignment", hioe);
943    }
944    // If an error above, fall-through to this simpler assign. Last resort.
945    return createAssignProcedures(hris);
946  }
947
948  /**
949   * Create round-robin assigns. Use on table creation to distribute out regions across cluster.
950   * @return AssignProcedures made out of the passed in <code>hris</code> and a call to the balancer
951   *         to populate the assigns with targets chosen using round-robin (default balancer
952   *         scheme). If at assign-time, the target chosen is no longer up, thats fine, the
953   *         AssignProcedure will ask the balancer for a new target, and so on.
954   */
955  public TransitRegionStateProcedure[] createRoundRobinAssignProcedures(List<RegionInfo> hris) {
956    return createRoundRobinAssignProcedures(hris, null);
957  }
958
959  static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
960    if (left.getRegion().isMetaRegion()) {
961      if (right.getRegion().isMetaRegion()) {
962        return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
963      }
964      return -1;
965    } else if (right.getRegion().isMetaRegion()) {
966      return +1;
967    }
968    if (left.getRegion().getTable().isSystemTable()) {
969      if (right.getRegion().getTable().isSystemTable()) {
970        return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
971      }
972      return -1;
973    } else if (right.getRegion().getTable().isSystemTable()) {
974      return +1;
975    }
976    return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
977  }
978
979  /**
980   * Create one TransitRegionStateProcedure to assign a region w/o specifying a target server. This
981   * method is called from HBCK2.
982   * @return an assign or null
983   */
984  public TransitRegionStateProcedure createOneAssignProcedure(RegionInfo ri, boolean override,
985    boolean force) {
986    TransitRegionStateProcedure trsp = null;
987    try {
988      trsp = createAssignProcedure(ri, null, override, force);
989    } catch (IOException ioe) {
990      LOG.info(
991        "Failed {} assign, override={}"
992          + (override ? "" : "; set override to by-pass state checks."),
993        ri.getEncodedName(), override, ioe);
994    }
995    return trsp;
996  }
997
998  /**
999   * Create one TransitRegionStateProcedure to unassign a region. This method is called from HBCK2.
1000   * @return an unassign or null
1001   */
1002  public TransitRegionStateProcedure createOneUnassignProcedure(RegionInfo ri, boolean override,
1003    boolean force) {
1004    RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(ri);
1005    TransitRegionStateProcedure trsp = null;
1006    regionNode.lock();
1007    try {
1008      if (override) {
1009        if (!force) {
1010          preTransitCheck(regionNode, STATES_EXPECTED_ON_UNASSIGN_OR_MOVE);
1011        }
1012        if (regionNode.getProcedure() != null) {
1013          regionNode.unsetProcedure(regionNode.getProcedure());
1014        }
1015      } else {
1016        // This is where we could throw an exception; i.e. override is false.
1017        preTransitCheck(regionNode, STATES_EXPECTED_ON_UNASSIGN_OR_MOVE);
1018      }
1019      assert regionNode.getProcedure() == null;
1020      trsp =
1021        TransitRegionStateProcedure.unassign(getProcedureEnvironment(), regionNode.getRegionInfo());
1022      regionNode.setProcedure(trsp);
1023    } catch (IOException ioe) {
1024      // 'override' must be false here.
1025      LOG.info("Failed {} unassign, override=false; set override to by-pass state checks.",
1026        ri.getEncodedName(), ioe);
1027    } finally {
1028      regionNode.unlock();
1029    }
1030    return trsp;
1031  }
1032
1033  /**
1034   * Create an array of TransitRegionStateProcedure w/o specifying a target server. Used as fallback
1035   * of caller is unable to do {@link #createAssignProcedures(Map)}.
1036   * <p/>
1037   * If no target server, at assign time, we will try to use the former location of the region if
1038   * one exists. This is how we 'retain' the old location across a server restart.
1039   * <p/>
1040   * Should only be called when you can make sure that no one can touch these regions other than
1041   * you. For example, when you are creating or enabling table. Presumes all Regions are in
1042   * appropriate state ripe for assign; no checking of Region state is done in here.
1043   * @see #createAssignProcedures(Map)
1044   */
1045  public TransitRegionStateProcedure[] createAssignProcedures(List<RegionInfo> hris) {
1046    return hris.stream().map(hri -> regionStates.getOrCreateRegionStateNode(hri))
1047      .map(regionNode -> createAssignProcedure(regionNode, null)).sorted(AssignmentManager::compare)
1048      .toArray(TransitRegionStateProcedure[]::new);
1049  }
1050
1051  /**
1052   * Tied to {@link #createAssignProcedures(List)} in that it is called if caller is unable to run
1053   * this method. Presumes all Regions are in appropriate state ripe for assign; no checking of
1054   * Region state is done in here.
1055   * @param assignments Map of assignments from which we produce an array of AssignProcedures.
1056   * @return Assignments made from the passed in <code>assignments</code>
1057   * @see #createAssignProcedures(List)
1058   */
1059  private TransitRegionStateProcedure[]
1060    createAssignProcedures(Map<ServerName, List<RegionInfo>> assignments) {
1061    return assignments.entrySet().stream()
1062      .flatMap(e -> e.getValue().stream().map(hri -> regionStates.getOrCreateRegionStateNode(hri))
1063        .map(regionNode -> createAssignProcedure(regionNode, e.getKey())))
1064      .sorted(AssignmentManager::compare).toArray(TransitRegionStateProcedure[]::new);
1065  }
1066
1067  // for creating unassign TRSP when disabling a table or closing excess region replicas
1068  private TransitRegionStateProcedure forceCreateUnssignProcedure(RegionStateNode regionNode) {
1069    regionNode.lock();
1070    try {
1071      if (regionNode.isInState(State.OFFLINE, State.CLOSED, State.SPLIT)) {
1072        return null;
1073      }
1074      // in general, a split parent should be in CLOSED or SPLIT state, but anyway, let's check it
1075      // here for safety
1076      if (regionNode.getRegionInfo().isSplit()) {
1077        LOG.warn("{} is a split parent but not in CLOSED or SPLIT state", regionNode);
1078        return null;
1079      }
1080      // As in DisableTableProcedure or ModifyTableProcedure, we will hold the xlock for table, so
1081      // we can make sure that this procedure has not been executed yet, as TRSP will hold the
1082      // shared lock for table all the time. So here we will unset it and when it is actually
1083      // executed, it will find that the attach procedure is not itself and quit immediately.
1084      if (regionNode.getProcedure() != null) {
1085        regionNode.unsetProcedure(regionNode.getProcedure());
1086      }
1087      return regionNode.setProcedure(TransitRegionStateProcedure.unassign(getProcedureEnvironment(),
1088        regionNode.getRegionInfo()));
1089    } finally {
1090      regionNode.unlock();
1091    }
1092  }
1093
1094  /**
1095   * Called by DisableTableProcedure to unassign all the regions for a table.
1096   */
1097  public TransitRegionStateProcedure[] createUnassignProceduresForDisabling(TableName tableName) {
1098    return regionStates.getTableRegionStateNodes(tableName).stream()
1099      .map(this::forceCreateUnssignProcedure).filter(p -> p != null)
1100      .toArray(TransitRegionStateProcedure[]::new);
1101  }
1102
1103  private int submitUnassignProcedure(TableName tableName,
1104    Function<RegionStateNode, Boolean> shouldSubmit, Consumer<RegionStateNode> logRIT,
1105    Consumer<TransitRegionStateProcedure> submit) {
1106    int inTransitionCount = 0;
1107    for (RegionStateNode regionNode : regionStates.getTableRegionStateNodes(tableName)) {
1108      regionNode.lock();
1109      try {
1110        if (shouldSubmit.apply(regionNode)) {
1111          if (regionNode.isTransitionScheduled()) {
1112            logRIT.accept(regionNode);
1113            inTransitionCount++;
1114            continue;
1115          }
1116          if (regionNode.isInState(State.OFFLINE, State.CLOSED, State.SPLIT)) {
1117            continue;
1118          }
1119          submit.accept(regionNode.setProcedure(TransitRegionStateProcedure
1120            .unassign(getProcedureEnvironment(), regionNode.getRegionInfo())));
1121        }
1122      } finally {
1123        regionNode.unlock();
1124      }
1125    }
1126    return inTransitionCount;
1127  }
1128
1129  /**
1130   * Called by DisableTableProcedure to unassign all regions for a table. Will skip submit unassign
1131   * procedure if the region is in transition, so you may need to call this method multiple times.
1132   * @param tableName the table for closing excess region replicas
1133   * @param submit    for submitting procedure
1134   * @return the number of regions in transition that we can not schedule unassign procedures
1135   */
1136  public int submitUnassignProcedureForDisablingTable(TableName tableName,
1137    Consumer<TransitRegionStateProcedure> submit) {
1138    return submitUnassignProcedure(tableName, rn -> true,
1139      rn -> LOG.debug("skip scheduling unassign procedure for {} when closing table regions "
1140        + "for disabling since it is in transition", rn),
1141      submit);
1142  }
1143
1144  /**
1145   * Called by ModifyTableProcedure to unassign all the excess region replicas for a table. Will
1146   * skip submit unassign procedure if the region is in transition, so you may need to call this
1147   * method multiple times.
1148   * @param tableName       the table for closing excess region replicas
1149   * @param newReplicaCount the new replica count, should be less than current replica count
1150   * @param submit          for submitting procedure
1151   * @return the number of regions in transition that we can not schedule unassign procedures
1152   */
1153  public int submitUnassignProcedureForClosingExcessRegionReplicas(TableName tableName,
1154    int newReplicaCount, Consumer<TransitRegionStateProcedure> submit) {
1155    return submitUnassignProcedure(tableName,
1156      rn -> rn.getRegionInfo().getReplicaId() >= newReplicaCount,
1157      rn -> LOG.debug("skip scheduling unassign procedure for {} when closing excess region "
1158        + "replicas since it is in transition", rn),
1159      submit);
1160  }
1161
1162  private int numberOfUnclosedRegions(TableName tableName,
1163    Function<RegionStateNode, Boolean> shouldSubmit) {
1164    int unclosed = 0;
1165    for (RegionStateNode regionNode : regionStates.getTableRegionStateNodes(tableName)) {
1166      regionNode.lock();
1167      try {
1168        if (shouldSubmit.apply(regionNode)) {
1169          if (!regionNode.isInState(State.OFFLINE, State.CLOSED, State.SPLIT)) {
1170            unclosed++;
1171          }
1172        }
1173      } finally {
1174        regionNode.unlock();
1175      }
1176    }
1177    return unclosed;
1178  }
1179
1180  public int numberOfUnclosedRegionsForDisabling(TableName tableName) {
1181    return numberOfUnclosedRegions(tableName, rn -> true);
1182  }
1183
1184  public int numberOfUnclosedExcessRegionReplicas(TableName tableName, int newReplicaCount) {
1185    return numberOfUnclosedRegions(tableName,
1186      rn -> rn.getRegionInfo().getReplicaId() >= newReplicaCount);
1187  }
1188
1189  public SplitTableRegionProcedure createSplitProcedure(final RegionInfo regionToSplit,
1190    final byte[] splitKey) throws IOException {
1191    return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
1192  }
1193
1194  public TruncateRegionProcedure createTruncateRegionProcedure(final RegionInfo regionToTruncate)
1195    throws IOException {
1196    return new TruncateRegionProcedure(getProcedureEnvironment(), regionToTruncate);
1197  }
1198
1199  public MergeTableRegionsProcedure createMergeProcedure(RegionInfo... ris) throws IOException {
1200    return new MergeTableRegionsProcedure(getProcedureEnvironment(), ris, false);
1201  }
1202
1203  /**
1204   * Delete the region states. This is called by "DeleteTable"
1205   */
1206  public void deleteTable(final TableName tableName) throws IOException {
1207    final ArrayList<RegionInfo> regions = regionStates.getTableRegionsInfo(tableName);
1208    regionStateStore.deleteRegions(regions);
1209    for (int i = 0; i < regions.size(); ++i) {
1210      final RegionInfo regionInfo = regions.get(i);
1211      regionStates.deleteRegion(regionInfo);
1212    }
1213  }
1214
1215  // ============================================================================================
1216  // RS Region Transition Report helpers
1217  // ============================================================================================
1218  private void reportRegionStateTransition(ReportRegionStateTransitionResponse.Builder builder,
1219    ServerStateNode serverNode, List<RegionStateTransition> transitionList) throws IOException {
1220    for (RegionStateTransition transition : transitionList) {
1221      switch (transition.getTransitionCode()) {
1222        case OPENED:
1223        case FAILED_OPEN:
1224        case CLOSED:
1225          assert transition.getRegionInfoCount() == 1 : transition;
1226          final RegionInfo hri = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
1227          long procId =
1228            transition.getProcIdCount() > 0 ? transition.getProcId(0) : Procedure.NO_PROC_ID;
1229          updateRegionTransition(serverNode, transition.getTransitionCode(), hri,
1230            transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM, procId);
1231          break;
1232        case READY_TO_SPLIT:
1233        case SPLIT:
1234        case SPLIT_REVERTED:
1235          assert transition.getRegionInfoCount() == 3 : transition;
1236          final RegionInfo parent = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
1237          final RegionInfo splitA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
1238          final RegionInfo splitB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
1239          updateRegionSplitTransition(serverNode, transition.getTransitionCode(), parent, splitA,
1240            splitB);
1241          break;
1242        case READY_TO_MERGE:
1243        case MERGED:
1244        case MERGE_REVERTED:
1245          assert transition.getRegionInfoCount() == 3 : transition;
1246          final RegionInfo merged = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
1247          final RegionInfo mergeA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
1248          final RegionInfo mergeB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
1249          updateRegionMergeTransition(serverNode, transition.getTransitionCode(), merged, mergeA,
1250            mergeB);
1251          break;
1252      }
1253    }
1254  }
1255
1256  public ReportRegionStateTransitionResponse reportRegionStateTransition(
1257    final ReportRegionStateTransitionRequest req) throws PleaseHoldException {
1258    ReportRegionStateTransitionResponse.Builder builder =
1259      ReportRegionStateTransitionResponse.newBuilder();
1260    ServerName serverName = ProtobufUtil.toServerName(req.getServer());
1261    ServerStateNode serverNode = regionStates.getServerNode(serverName);
1262    if (serverNode == null) {
1263      LOG.warn("No server node for {}", serverName);
1264      builder.setErrorMessage("No server node for " + serverName);
1265      return builder.build();
1266    }
1267    // here we have to acquire a read lock instead of a simple exclusive lock. This is because that
1268    // we should not block other reportRegionStateTransition call from the same region server. This
1269    // is not only about performance, but also to prevent dead lock. Think of the meta region is
1270    // also on the same region server and you hold the lock which blocks the
1271    // reportRegionStateTransition for meta, and since meta is not online, you will block inside the
1272    // lock protection to wait for meta online...
1273    serverNode.readLock().lock();
1274    try {
1275      // we only accept reportRegionStateTransition if the region server is online, see the comment
1276      // above in submitServerCrash method and HBASE-21508 for more details.
1277      if (serverNode.isInState(ServerState.ONLINE)) {
1278        try {
1279          reportRegionStateTransition(builder, serverNode, req.getTransitionList());
1280        } catch (PleaseHoldException e) {
1281          LOG.trace("Failed transition ", e);
1282          throw e;
1283        } catch (UnsupportedOperationException | IOException e) {
1284          // TODO: at the moment we have a single error message and the RS will abort
1285          // if the master says that one of the region transitions failed.
1286          LOG.warn("Failed transition", e);
1287          builder.setErrorMessage("Failed transition " + e.getMessage());
1288        }
1289      } else {
1290        LOG.warn("The region server {} is already dead, skip reportRegionStateTransition call",
1291          serverName);
1292        builder.setErrorMessage("You are dead");
1293      }
1294    } finally {
1295      serverNode.readLock().unlock();
1296    }
1297
1298    return builder.build();
1299  }
1300
1301  private void updateRegionTransition(ServerStateNode serverNode, TransitionCode state,
1302    RegionInfo regionInfo, long seqId, long procId) throws IOException {
1303    checkMetaLoaded(regionInfo);
1304
1305    RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
1306    if (regionNode == null) {
1307      // the table/region is gone. maybe a delete, split, merge
1308      throw new UnexpectedStateException(String.format(
1309        "Server %s was trying to transition region %s to %s. but Region is not known.",
1310        serverNode.getServerName(), regionInfo, state));
1311    }
1312    LOG.trace("Update region transition serverName={} region={} regionState={}",
1313      serverNode.getServerName(), regionNode, state);
1314
1315    regionNode.lock();
1316    try {
1317      if (!reportTransition(regionNode, serverNode, state, seqId, procId)) {
1318        // Don't log WARN if shutting down cluster; during shutdown. Avoid the below messages:
1319        // 2018-08-13 10:45:10,551 WARN ...AssignmentManager: No matching procedure found for
1320        // rit=OPEN, location=ve0538.halxg.cloudera.com,16020,1533493000958,
1321        // table=IntegrationTestBigLinkedList, region=65ab289e2fc1530df65f6c3d7cde7aa5 transition
1322        // to CLOSED
1323        // These happen because on cluster shutdown, we currently let the RegionServers close
1324        // regions. This is the only time that region close is not run by the Master (so cluster
1325        // goes down fast). Consider changing it so Master runs all shutdowns.
1326        if (
1327          this.master.getServerManager().isClusterShutdown() && state.equals(TransitionCode.CLOSED)
1328        ) {
1329          LOG.info("RegionServer {} {}", state, regionNode.getRegionInfo().getEncodedName());
1330        } else {
1331          LOG.warn("No matching procedure found for {} transition on {} to {}",
1332            serverNode.getServerName(), regionNode, state);
1333        }
1334      }
1335    } finally {
1336      regionNode.unlock();
1337    }
1338  }
1339
1340  private boolean reportTransition(RegionStateNode regionNode, ServerStateNode serverNode,
1341    TransitionCode state, long seqId, long procId) throws IOException {
1342    ServerName serverName = serverNode.getServerName();
1343    TransitRegionStateProcedure proc = regionNode.getProcedure();
1344    if (proc == null) {
1345      return false;
1346    }
1347    proc.reportTransition(master.getMasterProcedureExecutor().getEnvironment(), regionNode,
1348      serverName, state, seqId, procId);
1349    return true;
1350  }
1351
1352  private void updateRegionSplitTransition(final ServerStateNode serverNode,
1353    final TransitionCode state, final RegionInfo parent, final RegionInfo hriA,
1354    final RegionInfo hriB) throws IOException {
1355    checkMetaLoaded(parent);
1356
1357    if (state != TransitionCode.READY_TO_SPLIT) {
1358      throw new UnexpectedStateException(
1359        "unsupported split regionState=" + state + " for parent region " + parent
1360          + " maybe an old RS (< 2.0) had the operation in progress");
1361    }
1362
1363    // sanity check on the request
1364    if (!Bytes.equals(hriA.getEndKey(), hriB.getStartKey())) {
1365      throw new UnsupportedOperationException("unsupported split request with bad keys: parent="
1366        + parent + " hriA=" + hriA + " hriB=" + hriB);
1367    }
1368
1369    if (!master.isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
1370      LOG.warn("Split switch is off! skip split of " + parent);
1371      throw new DoNotRetryIOException(
1372        "Split region " + parent.getRegionNameAsString() + " failed due to split switch off");
1373    }
1374
1375    // Submit the Split procedure
1376    final byte[] splitKey = hriB.getStartKey();
1377    if (LOG.isDebugEnabled()) {
1378      LOG.debug("Split request from {}, parent={}, splitKey={}", serverNode.getServerName(), parent,
1379        Bytes.toStringBinary(splitKey));
1380    }
1381    // Processing this report happens asynchronously from other activities which can mutate
1382    // the region state. For example, a split procedure may already be running for this parent.
1383    // A split procedure cannot succeed if the parent region is no longer open, so we can
1384    // ignore it in that case.
1385    // Note that submitting more than one split procedure for a given region is
1386    // harmless -- the split is fenced in the procedure handling -- but it would be noisy in
1387    // the logs. Only one procedure can succeed. The other procedure(s) would abort during
1388    // initialization and report failure with WARN level logging.
1389    RegionState parentState = regionStates.getRegionState(parent);
1390    if (parentState != null && parentState.isOpened()) {
1391      master.getMasterProcedureExecutor().submitProcedure(createSplitProcedure(parent, splitKey));
1392    } else {
1393      LOG.info("Ignoring split request from {}, parent={} because parent is unknown or not open",
1394        serverNode.getServerName(), parent);
1395      return;
1396    }
1397
1398    // If the RS is < 2.0 throw an exception to abort the operation, we are handling the split
1399    if (master.getServerManager().getVersionNumber(serverNode.getServerName()) < 0x0200000) {
1400      throw new UnsupportedOperationException(
1401        String.format("Split handled by the master: " + "parent=%s hriA=%s hriB=%s",
1402          parent.getShortNameToLog(), hriA, hriB));
1403    }
1404  }
1405
1406  private void updateRegionMergeTransition(final ServerStateNode serverNode,
1407    final TransitionCode state, final RegionInfo merged, final RegionInfo hriA,
1408    final RegionInfo hriB) throws IOException {
1409    checkMetaLoaded(merged);
1410
1411    if (state != TransitionCode.READY_TO_MERGE) {
1412      throw new UnexpectedStateException(
1413        "Unsupported merge regionState=" + state + " for regionA=" + hriA + " regionB=" + hriB
1414          + " merged=" + merged + " maybe an old RS (< 2.0) had the operation in progress");
1415    }
1416
1417    if (!master.isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
1418      LOG.warn("Merge switch is off! skip merge of regionA=" + hriA + " regionB=" + hriB);
1419      throw new DoNotRetryIOException(
1420        "Merge of regionA=" + hriA + " regionB=" + hriB + " failed because merge switch is off");
1421    }
1422
1423    // Submit the Merge procedure
1424    if (LOG.isDebugEnabled()) {
1425      LOG.debug("Handling merge request from RS=" + merged + ", merged=" + merged);
1426    }
1427    master.getMasterProcedureExecutor().submitProcedure(createMergeProcedure(hriA, hriB));
1428
1429    // If the RS is < 2.0 throw an exception to abort the operation, we are handling the merge
1430    if (master.getServerManager().getVersionNumber(serverNode.getServerName()) < 0x0200000) {
1431      throw new UnsupportedOperationException(
1432        String.format("Merge not handled yet: regionState=%s merged=%s hriA=%s hriB=%s", state,
1433          merged, hriA, hriB));
1434    }
1435  }
1436
1437  // ============================================================================================
1438  // RS Status update (report online regions) helpers
1439  // ============================================================================================
1440  /**
1441   * The master will call this method when the RS send the regionServerReport(). The report will
1442   * contains the "online regions". This method will check the the online regions against the
1443   * in-memory state of the AM, and we will log a warn message if there is a mismatch. This is
1444   * because that there is no fencing between the reportRegionStateTransition method and
1445   * regionServerReport method, so there could be race and introduce inconsistency here, but
1446   * actually there is no problem.
1447   * <p/>
1448   * Please see HBASE-21421 and HBASE-21463 for more details.
1449   */
1450  public void reportOnlineRegions(ServerName serverName, Set<byte[]> regionNames) {
1451    if (!isRunning()) {
1452      return;
1453    }
1454    if (LOG.isTraceEnabled()) {
1455      LOG.trace("ReportOnlineRegions {} regionCount={}, metaLoaded={} {}", serverName,
1456        regionNames.size(), isMetaLoaded(),
1457        regionNames.stream().map(Bytes::toStringBinary).collect(Collectors.toList()));
1458    }
1459
1460    ServerStateNode serverNode = regionStates.getServerNode(serverName);
1461    if (serverNode == null) {
1462      LOG.warn("Got a report from server {} where its server node is null", serverName);
1463      return;
1464    }
1465    serverNode.readLock().lock();
1466    try {
1467      if (!serverNode.isInState(ServerState.ONLINE)) {
1468        LOG.warn("Got a report from a server result in state {}", serverNode);
1469        return;
1470      }
1471    } finally {
1472      serverNode.readLock().unlock();
1473    }
1474
1475    // Track the regionserver reported online regions in memory.
1476    synchronized (rsReports) {
1477      rsReports.put(serverName, regionNames);
1478    }
1479
1480    if (regionNames.isEmpty()) {
1481      // nothing to do if we don't have regions
1482      LOG.trace("no online region found on {}", serverName);
1483      return;
1484    }
1485    if (!isMetaLoaded()) {
1486      // we are still on startup, skip checking
1487      return;
1488    }
1489    // The Heartbeat tells us of what regions are on the region serve, check the state.
1490    checkOnlineRegionsReport(serverNode, regionNames);
1491  }
1492
1493  /**
1494   * Close <code>regionName</code> on <code>sn</code> silently and immediately without using a
1495   * Procedure or going via hbase:meta. For case where a RegionServer's hosting of a Region is not
1496   * aligned w/ the Master's accounting of Region state. This is for cleaning up an error in
1497   * accounting.
1498   */
1499  private void closeRegionSilently(ServerName sn, byte[] regionName) {
1500    try {
1501      RegionInfo ri = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
1502      // Pass -1 for timeout. Means do not wait.
1503      ServerManager.closeRegionSilentlyAndWait(this.master.getAsyncClusterConnection(), sn, ri, -1);
1504    } catch (Exception e) {
1505      LOG.error("Failed trying to close {} on {}", Bytes.toStringBinary(regionName), sn, e);
1506    }
1507  }
1508
1509  /**
1510   * Check that what the RegionServer reports aligns with the Master's image. If disagreement, we
1511   * will tell the RegionServer to expediently close a Region we do not think it should have.
1512   */
1513  private void checkOnlineRegionsReport(ServerStateNode serverNode, Set<byte[]> regionNames) {
1514    ServerName serverName = serverNode.getServerName();
1515    for (byte[] regionName : regionNames) {
1516      if (!isRunning()) {
1517        return;
1518      }
1519      RegionStateNode regionNode = regionStates.getRegionStateNodeFromName(regionName);
1520      if (regionNode == null) {
1521        String regionNameAsStr = Bytes.toStringBinary(regionName);
1522        LOG.warn("No RegionStateNode for {} but reported as up on {}; closing...", regionNameAsStr,
1523          serverName);
1524        closeRegionSilently(serverNode.getServerName(), regionName);
1525        continue;
1526      }
1527      final long lag = 1000;
1528      // This is just a fallback check designed to identify unexpected data inconsistencies, so we
1529      // use tryLock to attempt to acquire the lock, and if the lock cannot be acquired, we skip the
1530      // check. This will not cause any additional problems and also prevents the regionServerReport
1531      // call from being stuck for too long which may cause deadlock on region assignment.
1532      if (regionNode.tryLock()) {
1533        try {
1534          long diff = EnvironmentEdgeManager.currentTime() - regionNode.getLastUpdate();
1535          if (regionNode.isInState(State.OPENING, State.OPEN)) {
1536            // This is possible as a region server has just closed a region but the region server
1537            // report is generated before the closing, but arrive after the closing. Make sure
1538            // there
1539            // is some elapsed time so less false alarms.
1540            if (!regionNode.getRegionLocation().equals(serverName) && diff > lag) {
1541              LOG.warn("Reporting {} server does not match {} (time since last "
1542                + "update={}ms); closing...", serverName, regionNode, diff);
1543              closeRegionSilently(serverNode.getServerName(), regionName);
1544            }
1545          } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
1546            // So, we can get report that a region is CLOSED or SPLIT because a heartbeat
1547            // came in at about same time as a region transition. Make sure there is some
1548            // elapsed time so less false alarms.
1549            if (diff > lag) {
1550              LOG.warn("Reporting {} state does not match {} (time since last update={}ms)",
1551                serverName, regionNode, diff);
1552            }
1553          }
1554        } finally {
1555          regionNode.unlock();
1556        }
1557      } else {
1558        LOG.warn(
1559          "Unable to acquire lock for regionNode {}. It is likely that another thread is currently holding the lock. To avoid deadlock, skip execution for now.",
1560          regionNode);
1561      }
1562    }
1563  }
1564
1565  // ============================================================================================
1566  // RIT chore
1567  // ============================================================================================
1568  private static class RegionInTransitionChore extends ProcedureInMemoryChore<MasterProcedureEnv> {
1569    public RegionInTransitionChore(final int timeoutMsec) {
1570      super(timeoutMsec);
1571    }
1572
1573    @Override
1574    protected void periodicExecute(final MasterProcedureEnv env) {
1575      final AssignmentManager am = env.getAssignmentManager();
1576
1577      final RegionInTransitionStat ritStat = am.computeRegionInTransitionStat();
1578      if (ritStat.hasRegionsOverThreshold()) {
1579        for (RegionState hri : ritStat.getRegionOverThreshold()) {
1580          am.handleRegionOverStuckWarningThreshold(hri.getRegion());
1581        }
1582      }
1583
1584      // update metrics
1585      am.updateRegionsInTransitionMetrics(ritStat);
1586    }
1587  }
1588
1589  private static class DeadServerMetricRegionChore
1590    extends ProcedureInMemoryChore<MasterProcedureEnv> {
1591    public DeadServerMetricRegionChore(final int timeoutMsec) {
1592      super(timeoutMsec);
1593    }
1594
1595    @Override
1596    protected void periodicExecute(final MasterProcedureEnv env) {
1597      final ServerManager sm = env.getMasterServices().getServerManager();
1598      final AssignmentManager am = env.getAssignmentManager();
1599      // To minimize inconsistencies we are not going to snapshot live servers in advance in case
1600      // new servers are added; OTOH we don't want to add heavy sync for a consistent view since
1601      // this is for metrics. Instead, we're going to check each regions as we go; to avoid making
1602      // too many checks, we maintain a local lists of server, limiting us to false negatives. If
1603      // we miss some recently-dead server, we'll just see it next time.
1604      Set<ServerName> recentlyLiveServers = new HashSet<>();
1605      int deadRegions = 0, unknownRegions = 0;
1606      for (RegionStateNode rsn : am.getRegionStates().getRegionStateNodes()) {
1607        if (rsn.getState() != State.OPEN) {
1608          continue; // Opportunistic check, should quickly skip RITs, offline tables, etc.
1609        }
1610        // Do not need to acquire region state lock as this is only for showing metrics.
1611        ServerName sn = rsn.getRegionLocation();
1612        State state = rsn.getState();
1613        if (state != State.OPEN) {
1614          continue; // Mostly skipping RITs that are already being take care of.
1615        }
1616        if (sn == null) {
1617          ++unknownRegions; // Opened on null?
1618          continue;
1619        }
1620        if (recentlyLiveServers.contains(sn)) {
1621          continue;
1622        }
1623        ServerManager.ServerLiveState sls = sm.isServerKnownAndOnline(sn);
1624        switch (sls) {
1625          case LIVE:
1626            recentlyLiveServers.add(sn);
1627            break;
1628          case DEAD:
1629            ++deadRegions;
1630            break;
1631          case UNKNOWN:
1632            ++unknownRegions;
1633            break;
1634          default:
1635            throw new AssertionError("Unexpected " + sls);
1636        }
1637      }
1638      if (deadRegions > 0 || unknownRegions > 0) {
1639        LOG.info("Found {} OPEN regions on dead servers and {} OPEN regions on unknown servers",
1640          deadRegions, unknownRegions);
1641      }
1642
1643      am.updateDeadServerRegionMetrics(deadRegions, unknownRegions);
1644    }
1645  }
1646
1647  public RegionInTransitionStat computeRegionInTransitionStat() {
1648    final RegionInTransitionStat rit = new RegionInTransitionStat(getConfiguration());
1649    rit.update(this);
1650    return rit;
1651  }
1652
1653  public static class RegionInTransitionStat {
1654    private final int ritThreshold;
1655
1656    private HashMap<String, RegionState> ritsOverThreshold = null;
1657    private long statTimestamp;
1658    private long oldestRITTime = 0;
1659    private int totalRITsTwiceThreshold = 0;
1660    private int totalRITs = 0;
1661
1662    public RegionInTransitionStat(final Configuration conf) {
1663      this.ritThreshold =
1664        conf.getInt(METRICS_RIT_STUCK_WARNING_THRESHOLD, DEFAULT_RIT_STUCK_WARNING_THRESHOLD);
1665    }
1666
1667    public int getRITThreshold() {
1668      return ritThreshold;
1669    }
1670
1671    public long getTimestamp() {
1672      return statTimestamp;
1673    }
1674
1675    public int getTotalRITs() {
1676      return totalRITs;
1677    }
1678
1679    public long getOldestRITTime() {
1680      return oldestRITTime;
1681    }
1682
1683    public int getTotalRITsOverThreshold() {
1684      Map<String, RegionState> m = this.ritsOverThreshold;
1685      return m != null ? m.size() : 0;
1686    }
1687
1688    public boolean hasRegionsTwiceOverThreshold() {
1689      return totalRITsTwiceThreshold > 0;
1690    }
1691
1692    public boolean hasRegionsOverThreshold() {
1693      Map<String, RegionState> m = this.ritsOverThreshold;
1694      return m != null && !m.isEmpty();
1695    }
1696
1697    public Collection<RegionState> getRegionOverThreshold() {
1698      Map<String, RegionState> m = this.ritsOverThreshold;
1699      return m != null ? m.values() : Collections.emptySet();
1700    }
1701
1702    public boolean isRegionOverThreshold(final RegionInfo regionInfo) {
1703      Map<String, RegionState> m = this.ritsOverThreshold;
1704      return m != null && m.containsKey(regionInfo.getEncodedName());
1705    }
1706
1707    public boolean isRegionTwiceOverThreshold(final RegionInfo regionInfo) {
1708      Map<String, RegionState> m = this.ritsOverThreshold;
1709      if (m == null) {
1710        return false;
1711      }
1712      final RegionState state = m.get(regionInfo.getEncodedName());
1713      if (state == null) {
1714        return false;
1715      }
1716      return (statTimestamp - state.getStamp()) > (ritThreshold * 2);
1717    }
1718
1719    protected void update(final AssignmentManager am) {
1720      this.statTimestamp = EnvironmentEdgeManager.currentTime();
1721      update(am.getRegionsStateInTransition(), statTimestamp);
1722
1723      if (LOG.isDebugEnabled() && ritsOverThreshold != null && !ritsOverThreshold.isEmpty()) {
1724        LOG.debug("RITs over threshold: {}",
1725          ritsOverThreshold.entrySet().stream()
1726            .map(e -> e.getKey() + ":" + e.getValue().getState().name())
1727            .collect(Collectors.joining("\n")));
1728      }
1729    }
1730
1731    private void update(final Collection<RegionState> regions, final long currentTime) {
1732      for (RegionState state : regions) {
1733        totalRITs++;
1734        final long ritStartedMs = state.getStamp();
1735        if (ritStartedMs == 0) {
1736          // Don't output bogus values to metrics if they accidentally make it here.
1737          LOG.warn("The RIT {} has no start time", state.getRegion());
1738          continue;
1739        }
1740        final long ritTime = currentTime - ritStartedMs;
1741        if (ritTime > ritThreshold) {
1742          if (ritsOverThreshold == null) {
1743            ritsOverThreshold = new HashMap<String, RegionState>();
1744          }
1745          ritsOverThreshold.put(state.getRegion().getEncodedName(), state);
1746          totalRITsTwiceThreshold += (ritTime > (ritThreshold * 2)) ? 1 : 0;
1747        }
1748        if (oldestRITTime < ritTime) {
1749          oldestRITTime = ritTime;
1750        }
1751      }
1752    }
1753  }
1754
1755  private void updateRegionsInTransitionMetrics(final RegionInTransitionStat ritStat) {
1756    metrics.updateRITOldestAge(ritStat.getOldestRITTime());
1757    metrics.updateRITCount(ritStat.getTotalRITs());
1758    metrics.updateRITCountOverThreshold(ritStat.getTotalRITsOverThreshold());
1759  }
1760
1761  private void updateDeadServerRegionMetrics(int deadRegions, int unknownRegions) {
1762    metrics.updateDeadServerOpenRegions(deadRegions);
1763    metrics.updateUnknownServerOpenRegions(unknownRegions);
1764  }
1765
1766  private void handleRegionOverStuckWarningThreshold(final RegionInfo regionInfo) {
1767    final RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
1768    // if (regionNode.isStuck()) {
1769    LOG.warn("STUCK Region-In-Transition {}", regionNode);
1770  }
1771
1772  // ============================================================================================
1773  // TODO: Master load/bootstrap
1774  // ============================================================================================
1775  public void joinCluster() throws IOException {
1776    long startTime = System.nanoTime();
1777    LOG.debug("Joining cluster...");
1778
1779    // Scan hbase:meta to build list of existing regions, servers, and assignment.
1780    // hbase:meta is online now or will be. Inside loadMeta, we keep trying. Can't make progress
1781    // w/o meta.
1782    loadMeta();
1783
1784    while (master.getServerManager().countOfRegionServers() < 1) {
1785      LOG.info("Waiting for RegionServers to join; current count={}",
1786        master.getServerManager().countOfRegionServers());
1787      Threads.sleep(250);
1788    }
1789    LOG.info("Number of RegionServers={}", master.getServerManager().countOfRegionServers());
1790
1791    // Start the chores
1792    master.getMasterProcedureExecutor().addChore(this.ritChore);
1793    master.getMasterProcedureExecutor().addChore(this.deadMetricChore);
1794
1795    long costMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
1796    LOG.info("Joined the cluster in {}", StringUtils.humanTimeDiff(costMs));
1797  }
1798
1799  /**
1800   * Create assign procedure for offline regions. Just follow the old
1801   * processofflineServersWithOnlineRegions method. Since now we do not need to deal with dead
1802   * server any more, we only deal with the regions in OFFLINE state in this method. And this is a
1803   * bit strange, that for new regions, we will add it in CLOSED state instead of OFFLINE state, and
1804   * usually there will be a procedure to track them. The processofflineServersWithOnlineRegions is
1805   * a legacy from long ago, as things are going really different now, maybe we do not need this
1806   * method any more. Need to revisit later.
1807   */
1808  // Public so can be run by the Master as part of the startup. Needs hbase:meta to be online.
1809  // Needs to be done after the table state manager has been started.
1810  public void processOfflineRegions() {
1811    TransitRegionStateProcedure[] procs =
1812      regionStates.getRegionStateNodes().stream().filter(rsn -> rsn.isInState(State.OFFLINE))
1813        .filter(rsn -> isTableEnabled(rsn.getRegionInfo().getTable())).map(rsn -> {
1814          rsn.lock();
1815          try {
1816            if (rsn.getProcedure() != null) {
1817              return null;
1818            } else {
1819              return rsn.setProcedure(TransitRegionStateProcedure.assign(getProcedureEnvironment(),
1820                rsn.getRegionInfo(), null));
1821            }
1822          } finally {
1823            rsn.unlock();
1824          }
1825        }).filter(p -> p != null).toArray(TransitRegionStateProcedure[]::new);
1826    if (procs.length > 0) {
1827      master.getMasterProcedureExecutor().submitProcedures(procs);
1828    }
1829  }
1830
1831  /*
1832   * AM internal RegionStateStore.RegionStateVisitor implementation. To be used when scanning META
1833   * table for region rows, using RegionStateStore utility methods. RegionStateStore methods will
1834   * convert Result into proper RegionInfo instances, but those would still need to be added into
1835   * AssignmentManager.regionStates in-memory cache. RegionMetaLoadingVisitor.visitRegionState
1836   * method provides the logic for adding RegionInfo instances as loaded from latest META scan into
1837   * AssignmentManager.regionStates.
1838   */
1839  private class RegionMetaLoadingVisitor implements RegionStateStore.RegionStateVisitor {
1840
1841    @Override
1842    public void visitRegionState(Result result, final RegionInfo regionInfo, final State state,
1843      final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
1844      if (
1845        state == null && regionLocation == null && lastHost == null
1846          && openSeqNum == SequenceId.NO_SEQUENCE_ID
1847      ) {
1848        // This is a row with nothing in it.
1849        LOG.warn("Skipping empty row={}", result);
1850        return;
1851      }
1852      State localState = state;
1853      if (localState == null) {
1854        // No region state column data in hbase:meta table! Are I doing a rolling upgrade from
1855        // hbase1 to hbase2? Am I restoring a SNAPSHOT or otherwise adding a region to hbase:meta?
1856        // In any of these cases, state is empty. For now, presume OFFLINE but there are probably
1857        // cases where we need to probe more to be sure this correct; TODO informed by experience.
1858        LOG.info(regionInfo.getEncodedName() + " regionState=null; presuming " + State.OFFLINE);
1859        localState = State.OFFLINE;
1860      }
1861      RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
1862      // Do not need to lock on regionNode, as we can make sure that before we finish loading
1863      // meta, all the related procedures can not be executed. The only exception is for meta
1864      // region related operations, but here we do not load the informations for meta region.
1865      regionNode.setState(localState);
1866      regionNode.setLastHost(lastHost);
1867      regionNode.setRegionLocation(regionLocation);
1868      regionNode.setOpenSeqNum(openSeqNum);
1869
1870      // Note: keep consistent with other methods, see region(Opening|Opened|Closing)
1871      // RIT/ServerCrash handling should take care of the transiting regions.
1872      if (
1873        localState.matches(State.OPEN, State.OPENING, State.CLOSING, State.SPLITTING, State.MERGING)
1874      ) {
1875        assert regionLocation != null : "found null region location for " + regionNode;
1876        // TODO: this could lead to some orphan server state nodes, as it is possible that the
1877        // region server is already dead and its SCP has already finished but we have
1878        // persisted an opening state on this region server. Finally the TRSP will assign the
1879        // region to another region server, so it will not cause critical problems, just waste
1880        // some memory as no one will try to cleanup these orphan server state nodes.
1881        regionStates.createServer(regionLocation);
1882        regionStates.addRegionToServer(regionNode);
1883      } else if (localState == State.OFFLINE || regionInfo.isOffline()) {
1884        regionStates.addToOfflineRegions(regionNode);
1885      }
1886      if (regionNode.getProcedure() != null) {
1887        regionNode.getProcedure().stateLoaded(AssignmentManager.this, regionNode);
1888      }
1889      // add regions to RIT while visiting the meta
1890      regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
1891      // If region location of region belongs to a dead server mark the region crashed
1892      if (
1893        regionNode.getRegionLocation() != null
1894          && master.getServerManager().isServerDead(regionNode.getRegionLocation())
1895      ) {
1896        long timeOfCrash = master.getServerManager().getDeadServers()
1897          .getDeathTimestamp(regionNode.getRegionLocation());
1898        if (timeOfCrash != 0) {
1899          regionNode.crashed(timeOfCrash);
1900        }
1901        regionInTransitionTracker.regionCrashed(regionNode);
1902      }
1903    }
1904  };
1905
1906  /**
1907   * Attempt to load {@code regionInfo} from META, adding any results to the
1908   * {@link #regionStateStore} Is NOT aware of replica regions.
1909   * @param regionInfo the region to be loaded from META.
1910   * @throws IOException If some error occurs while querying META or parsing results.
1911   */
1912  public void populateRegionStatesFromMeta(@NonNull final RegionInfo regionInfo)
1913    throws IOException {
1914    final String regionEncodedName = RegionInfo.DEFAULT_REPLICA_ID == regionInfo.getReplicaId()
1915      ? regionInfo.getEncodedName()
1916      : RegionInfoBuilder.newBuilder(regionInfo).setReplicaId(RegionInfo.DEFAULT_REPLICA_ID).build()
1917        .getEncodedName();
1918    populateRegionStatesFromMeta(regionEncodedName);
1919  }
1920
1921  /**
1922   * Attempt to load {@code regionEncodedName} from META, adding any results to the
1923   * {@link #regionStateStore} Is NOT aware of replica regions.
1924   * @param regionEncodedName encoded name for the region to be loaded from META.
1925   * @throws IOException If some error occurs while querying META or parsing results.
1926   */
1927  public void populateRegionStatesFromMeta(@NonNull String regionEncodedName) throws IOException {
1928    final RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
1929    regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
1930  }
1931
1932  private void loadMeta() throws IOException {
1933    // TODO: use a thread pool
1934    regionStateStore.visitMeta(new RegionMetaLoadingVisitor());
1935  }
1936
1937  /**
1938   * Used to check if the meta loading is done.
1939   * <p/>
1940   * if not we throw PleaseHoldException since we are rebuilding the RegionStates
1941   * @param hri region to check if it is already rebuild
1942   * @throws PleaseHoldException if meta has not been loaded yet
1943   */
1944  private void checkMetaLoaded(RegionInfo hri) throws PleaseHoldException {
1945    if (!isRunning()) {
1946      throw new PleaseHoldException("AssignmentManager not running");
1947    }
1948    boolean meta = isMetaRegion(hri);
1949    boolean metaLoaded = isMetaLoaded();
1950    if (!meta && !metaLoaded) {
1951      throw new PleaseHoldException(
1952        "Master not fully online; hbase:meta=" + meta + ", metaLoaded=" + metaLoaded);
1953    }
1954  }
1955
1956  // ============================================================================================
1957  // TODO: Metrics
1958  // ============================================================================================
1959  public int getNumRegionsOpened() {
1960    // TODO: Used by TestRegionPlacement.java and assume monotonically increasing value
1961    return 0;
1962  }
1963
1964  /**
1965   * Usually run by the Master in reaction to server crash during normal processing. Can also be
1966   * invoked via external RPC to effect repair; in the latter case, the 'force' flag is set so we
1967   * push through the SCP though context may indicate already-running-SCP (An old SCP may have
1968   * exited abnormally, or damaged cluster may still have references in hbase:meta to 'Unknown
1969   * Servers' -- servers that are not online or in dead servers list, etc.)
1970   * @param force Set if the request came in externally over RPC (via hbck2). Force means run the
1971   *              SCP even if it seems as though there might be an outstanding SCP running.
1972   * @return pid of scheduled SCP or {@link Procedure#NO_PROC_ID} if none scheduled.
1973   */
1974  public long submitServerCrash(ServerName serverName, boolean shouldSplitWal, boolean force) {
1975    // May be an 'Unknown Server' so handle case where serverNode is null.
1976    ServerStateNode serverNode = regionStates.getServerNode(serverName);
1977    // Remove the in-memory rsReports result
1978    synchronized (rsReports) {
1979      rsReports.remove(serverName);
1980    }
1981    if (serverNode == null) {
1982      if (force) {
1983        LOG.info("Force adding ServerCrashProcedure for {} when server node is null", serverName);
1984      } else {
1985        // for normal case, do not schedule SCP if ServerStateNode is null
1986        LOG.warn("Skip adding ServerCrashProcedure for {} because server node is null", serverName);
1987        return Procedure.NO_PROC_ID;
1988      }
1989    }
1990
1991    ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
1992    // We hold the write lock here for fencing on reportRegionStateTransition. Once we set the
1993    // server state to CRASHED, we will no longer accept the reportRegionStateTransition call from
1994    // this server. This is used to simplify the implementation for TRSP and SCP, where we can make
1995    // sure that, the region list fetched by SCP will not be changed any more.
1996    if (serverNode != null) {
1997      serverNode.writeLock().lock();
1998    }
1999    try {
2000
2001      boolean carryingMeta = isCarryingMeta(serverName);
2002      if (serverNode != null && !serverNode.isInState(ServerState.ONLINE)) {
2003        if (force) {
2004          LOG.info("Force adding ServerCrashProcedure for {} (meta={}) when state is not {}",
2005            serverNode, carryingMeta, ServerState.ONLINE);
2006        } else {
2007          LOG.info("Skip adding ServerCrashProcedure for {} (meta={}) when state is not {}",
2008            serverNode, carryingMeta, ServerState.ONLINE);
2009          return Procedure.NO_PROC_ID;
2010        }
2011      }
2012      MasterProcedureEnv mpe = procExec.getEnvironment();
2013      // If serverNode == null, then 'Unknown Server'. Schedule HBCKSCP instead.
2014      // HBCKSCP scours Master in-memory state AND hbase;meta for references to
2015      // serverName just-in-case. An SCP that is scheduled when the server is
2016      // 'Unknown' probably originated externally with HBCK2 fix-it tool.
2017      ServerState oldState = null;
2018      if (serverNode != null) {
2019        oldState = serverNode.getState();
2020        serverNode.setState(ServerState.CRASHED);
2021      }
2022      ServerCrashProcedure scp = force
2023        ? new HBCKServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta)
2024        : new ServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta);
2025      long pid = procExec.submitProcedure(scp);
2026      LOG.info("Scheduled ServerCrashProcedure pid={} for {} (carryingMeta={}){}.", pid, serverName,
2027        carryingMeta,
2028        serverNode == null ? "" : " " + serverNode.toString() + ", oldState=" + oldState);
2029      return pid;
2030    } finally {
2031      if (serverNode != null) {
2032        serverNode.writeLock().unlock();
2033      }
2034    }
2035  }
2036
2037  public void offlineRegion(final RegionInfo regionInfo) {
2038    // TODO used by MasterRpcServices
2039    RegionStateNode node = regionStates.getRegionStateNode(regionInfo);
2040    if (node != null) {
2041      node.offline();
2042    }
2043  }
2044
2045  public void onlineRegion(final RegionInfo regionInfo, final ServerName serverName) {
2046    // TODO used by TestSplitTransactionOnCluster.java
2047  }
2048
2049  public Map<ServerName, List<RegionInfo>>
2050    getSnapShotOfAssignment(final Collection<RegionInfo> regions) {
2051    return regionStates.getSnapShotOfAssignment(regions);
2052  }
2053
2054  // ============================================================================================
2055  // TODO: UTILS/HELPERS?
2056  // ============================================================================================
2057  /**
2058   * Used by the client (via master) to identify if all regions have the schema updates
2059   * @return Pair indicating the status of the alter command (pending/total)
2060   */
2061  public Pair<Integer, Integer> getReopenStatus(TableName tableName) {
2062    if (isTableDisabled(tableName)) {
2063      return new Pair<Integer, Integer>(0, 0);
2064    }
2065
2066    final List<RegionState> states = regionStates.getTableRegionStates(tableName);
2067    int ritCount = 0;
2068    for (RegionState regionState : states) {
2069      if (!regionState.isOpened() && !regionState.isSplit()) {
2070        ritCount++;
2071      }
2072    }
2073    return new Pair<Integer, Integer>(ritCount, states.size());
2074  }
2075
2076  // This comparator sorts the RegionStates by time stamp then Region name.
2077  // Comparing by timestamp alone can lead us to discard different RegionStates that happen
2078  // to share a timestamp.
2079  private final static class RegionStateStampComparator implements Comparator<RegionState> {
2080    @Override
2081    public int compare(final RegionState l, final RegionState r) {
2082      int stampCmp = Long.compare(l.getStamp(), r.getStamp());
2083      return stampCmp != 0 ? stampCmp : RegionInfo.COMPARATOR.compare(l.getRegion(), r.getRegion());
2084    }
2085  }
2086
2087  public final static RegionStateStampComparator REGION_STATE_STAMP_COMPARATOR =
2088    new RegionStateStampComparator();
2089
2090  // ============================================================================================
2091  // TODO: Region State In Transition
2092  // ============================================================================================
2093  public boolean hasRegionsInTransition() {
2094    return regionInTransitionTracker.hasRegionsInTransition();
2095  }
2096
2097  public List<RegionStateNode> getRegionsInTransition() {
2098    return regionInTransitionTracker.getRegionsInTransition();
2099  }
2100
2101  public boolean isRegionInTransition(final RegionInfo regionInfo) {
2102    return regionInTransitionTracker.isRegionInTransition(regionInfo);
2103  }
2104
2105  public int getRegionTransitScheduledCount() {
2106    return regionStates.getRegionTransitScheduledCount();
2107  }
2108
2109  /**
2110   * Get the number of regions in transition.
2111   */
2112  public int getRegionsInTransitionCount() {
2113    return regionInTransitionTracker.getRegionsInTransition().size();
2114  }
2115
2116  public SortedSet<RegionState> getRegionsStateInTransition() {
2117    final SortedSet<RegionState> rit = new TreeSet<RegionState>(REGION_STATE_STAMP_COMPARATOR);
2118    for (RegionStateNode node : getRegionsInTransition()) {
2119      rit.add(node.toRegionState());
2120    }
2121    return rit;
2122  }
2123
2124  public List<RegionInfo> getAssignedRegions() {
2125    return regionStates.getAssignedRegions();
2126  }
2127
2128  /**
2129   * Resolve a cached {@link RegionInfo} from the region name as a {@code byte[]}.
2130   */
2131  public RegionInfo getRegionInfo(final byte[] regionName) {
2132    final RegionStateNode regionState = regionStates.getRegionStateNodeFromName(regionName);
2133    return regionState != null ? regionState.getRegionInfo() : null;
2134  }
2135
2136  /**
2137   * Resolve a cached {@link RegionInfo} from the encoded region name as a {@code String}.
2138   */
2139  public RegionInfo getRegionInfo(final String encodedRegionName) {
2140    final RegionStateNode regionState =
2141      regionStates.getRegionStateNodeFromEncodedRegionName(encodedRegionName);
2142    return regionState != null ? regionState.getRegionInfo() : null;
2143  }
2144
2145  // ============================================================================================
2146  // Expected states on region state transition.
2147  // Notice that there is expected states for transiting to OPENING state, this is because SCP.
2148  // See the comments in regionOpening method for more details.
2149  // ============================================================================================
2150  private static final State[] STATES_EXPECTED_ON_OPEN = { State.OPENING, // Normal case
2151    State.OPEN // Retrying
2152  };
2153
2154  private static final State[] STATES_EXPECTED_ON_CLOSING = { State.OPEN, // Normal case
2155    State.CLOSING, // Retrying
2156    State.SPLITTING, // Offline the split parent
2157    State.MERGING // Offline the merge parents
2158  };
2159
2160  private static final State[] STATES_EXPECTED_ON_CLOSED = { State.CLOSING, // Normal case
2161    State.CLOSED // Retrying
2162  };
2163
2164  // This is for manually scheduled region assign, can add other states later if we find out other
2165  // usages
2166  private static final State[] STATES_EXPECTED_ON_ASSIGN = { State.CLOSED, State.OFFLINE };
2167
2168  // We only allow unassign or move a region which is in OPEN state.
2169  private static final State[] STATES_EXPECTED_ON_UNASSIGN_OR_MOVE = { State.OPEN };
2170
2171  // ============================================================================================
2172  // Region Status update
2173  // Should only be called in TransitRegionStateProcedure(and related procedures), as the locking
2174  // and pre-assumptions are very tricky.
2175  // ============================================================================================
2176  private CompletableFuture<Void> transitStateAndUpdate(RegionStateNode regionNode,
2177    RegionState.State newState, RegionState.State... expectedStates) {
2178    RegionState.State state = regionNode.getState();
2179    try {
2180      regionNode.transitionState(newState, expectedStates);
2181    } catch (UnexpectedStateException e) {
2182      return FutureUtils.failedFuture(e);
2183    }
2184    CompletableFuture<Void> future = regionStateStore.updateRegionLocation(regionNode);
2185    FutureUtils.addListener(future, (r, e) -> {
2186      if (e != null) {
2187        // revert
2188        regionNode.setState(state);
2189      } else {
2190        regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
2191      }
2192    });
2193    return future;
2194  }
2195
2196  // should be called within the synchronized block of RegionStateNode
2197  CompletableFuture<Void> regionOpening(RegionStateNode regionNode) {
2198    // As in SCP, for performance reason, there is no TRSP attached with this region, we will not
2199    // update the region state, which means that the region could be in any state when we want to
2200    // assign it after a RS crash. So here we do not pass the expectedStates parameter.
2201    return transitStateAndUpdate(regionNode, State.OPENING).thenAccept(r -> {
2202      ServerStateNode serverNode = regionStates.getServerNode(regionNode.getRegionLocation());
2203      // Here the server node could be null. For example, we want to assign the region to a given
2204      // region server and it crashes, and it is the region server which holds hbase:meta, then the
2205      // above transitStateAndUpdate call will never succeed until we finishes the SCP for it. But
2206      // after the SCP finishes, the server node will be removed, so when we arrive there, the
2207      // server
2208      // node will be null. This is not a big problem if we skip adding it, as later we will fail to
2209      // execute the remote procedure on the region server and then try to assign to another region
2210      // server
2211      if (serverNode != null) {
2212        serverNode.addRegion(regionNode);
2213      }
2214      // update the operation count metrics
2215      metrics.incrementOperationCounter();
2216    });
2217  }
2218
2219  // should be called under the RegionStateNode lock
2220  // The parameter 'giveUp' means whether we will try to open the region again, if it is true, then
2221  // we will persist the FAILED_OPEN state into hbase:meta.
2222  CompletableFuture<Void> regionFailedOpen(RegionStateNode regionNode, boolean giveUp) {
2223    RegionState.State state = regionNode.getState();
2224    ServerName regionLocation = regionNode.getRegionLocation();
2225    if (!giveUp) {
2226      if (regionLocation != null) {
2227        regionStates.removeRegionFromServer(regionLocation, regionNode);
2228      }
2229      return CompletableFuture.completedFuture(null);
2230    }
2231    regionNode.setState(State.FAILED_OPEN);
2232    regionNode.setRegionLocation(null);
2233    CompletableFuture<Void> future = regionStateStore.updateRegionLocation(regionNode);
2234    FutureUtils.addListener(future, (r, e) -> {
2235      if (e == null) {
2236        if (regionLocation != null) {
2237          regionStates.removeRegionFromServer(regionLocation, regionNode);
2238        }
2239        regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
2240      } else {
2241        // revert
2242        regionNode.setState(state);
2243        regionNode.setRegionLocation(regionLocation);
2244      }
2245    });
2246    return future;
2247  }
2248
2249  // should be called under the RegionStateNode lock
2250  CompletableFuture<Void> regionClosing(RegionStateNode regionNode) {
2251    return transitStateAndUpdate(regionNode, State.CLOSING, STATES_EXPECTED_ON_CLOSING)
2252      .thenAccept(r -> {
2253        RegionInfo hri = regionNode.getRegionInfo();
2254        // Set meta has not initialized early. so people trying to create/edit tables will wait
2255        if (isMetaRegion(hri)) {
2256          setMetaAssigned(hri, false);
2257        }
2258        // update the operation count metrics
2259        metrics.incrementOperationCounter();
2260      });
2261  }
2262
2263  // for open and close, they will first be persist to the procedure store in
2264  // RegionRemoteProcedureBase. So here we will first change the in memory state as it is considered
2265  // as succeeded if the persistence to procedure store is succeeded, and then when the
2266  // RegionRemoteProcedureBase is woken up, we will persist the RegionStateNode to hbase:meta.
2267
2268  // should be called under the RegionStateNode lock
2269  void regionOpenedWithoutPersistingToMeta(RegionStateNode regionNode)
2270    throws UnexpectedStateException {
2271    regionNode.transitionState(State.OPEN, STATES_EXPECTED_ON_OPEN);
2272    RegionInfo regionInfo = regionNode.getRegionInfo();
2273    regionStates.addRegionToServer(regionNode);
2274    regionStates.removeFromFailedOpen(regionInfo);
2275  }
2276
2277  // should be called under the RegionStateNode lock
2278  void regionClosedWithoutPersistingToMeta(RegionStateNode regionNode)
2279    throws UnexpectedStateException {
2280    ServerName regionLocation = regionNode.getRegionLocation();
2281    regionNode.transitionState(State.CLOSED, STATES_EXPECTED_ON_CLOSED);
2282    regionNode.setRegionLocation(null);
2283    if (regionLocation != null) {
2284      regionNode.setLastHost(regionLocation);
2285      regionStates.removeRegionFromServer(regionLocation, regionNode);
2286    }
2287  }
2288
2289  // should be called under the RegionStateNode lock
2290  CompletableFuture<Void> persistToMeta(RegionStateNode regionNode) {
2291    return regionStateStore.updateRegionLocation(regionNode).thenAccept(r -> {
2292      RegionInfo regionInfo = regionNode.getRegionInfo();
2293      if (isMetaRegion(regionInfo) && regionNode.getState() == State.OPEN) {
2294        // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
2295        // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
2296        // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
2297        // on table that contains state.
2298        setMetaAssigned(regionInfo, true);
2299      }
2300      regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
2301    });
2302  }
2303
2304  // should be called under the RegionStateNode lock
2305  // for SCP
2306  public CompletableFuture<Void> regionClosedAbnormally(RegionStateNode regionNode) {
2307    RegionState.State state = regionNode.getState();
2308    ServerName regionLocation = regionNode.getRegionLocation();
2309    regionNode.setState(State.ABNORMALLY_CLOSED);
2310    regionNode.setRegionLocation(null);
2311    CompletableFuture<Void> future = regionStateStore.updateRegionLocation(regionNode);
2312    FutureUtils.addListener(future, (r, e) -> {
2313      if (e == null) {
2314        if (regionLocation != null) {
2315          regionNode.setLastHost(regionLocation);
2316          regionStates.removeRegionFromServer(regionLocation, regionNode);
2317        }
2318        regionInTransitionTracker.handleRegionStateNodeOperation(regionNode);
2319      } else {
2320        // revert
2321        regionNode.setState(state);
2322        regionNode.setRegionLocation(regionLocation);
2323      }
2324    });
2325    return future;
2326  }
2327
2328  // ============================================================================================
2329  // The above methods can only be called in TransitRegionStateProcedure(and related procedures)
2330  // ============================================================================================
2331
2332  // As soon as a server a crashed, region hosting on that are un-available, this method helps to
2333  // track those un-available regions. This method can only be called from ServerCrashProcedure.
2334  public void markRegionsAsCrashed(List<RegionInfo> regionsOnCrashedServer,
2335    ServerCrashProcedure scp) {
2336    ServerName crashedServerName = scp.getServerName();
2337    for (RegionInfo regionInfo : regionsOnCrashedServer) {
2338      RegionStateNode node = regionStates.getOrCreateRegionStateNode(regionInfo);
2339      if (node.getRegionLocation() == crashedServerName) {
2340        node.crashed(scp.getSubmittedTime());
2341        regionInTransitionTracker.regionCrashed(node);
2342      }
2343    }
2344  }
2345
2346  public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
2347    final RegionInfo daughterA, final RegionInfo daughterB) throws IOException {
2348    // Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
2349    // The parent stays in regionStates until cleared when removed by CatalogJanitor.
2350    // Update its state in regionStates to it shows as offline and split when read
2351    // later figuring what regions are in a table and what are not: see
2352    // regionStates#getRegionsOfTable
2353    final RegionStateNode node = regionStates.getOrCreateRegionStateNode(parent);
2354    node.setState(State.SPLIT);
2355    final RegionStateNode nodeA = regionStates.getOrCreateRegionStateNode(daughterA);
2356    nodeA.setState(State.SPLITTING_NEW);
2357    final RegionStateNode nodeB = regionStates.getOrCreateRegionStateNode(daughterB);
2358    nodeB.setState(State.SPLITTING_NEW);
2359
2360    TableDescriptor td = master.getTableDescriptors().get(parent.getTable());
2361    // TODO: here we just update the parent region info in meta, to set split and offline to true,
2362    // without changing the one in the region node. This is a bit confusing but the region info
2363    // field in RegionStateNode is not expected to be changed in the current design. Need to find a
2364    // possible way to address this problem, or at least adding more comments about the trick to
2365    // deal with this problem, that when you want to filter out split parent, you need to check both
2366    // the RegionState on whether it is split, and also the region info. If one of them matches then
2367    // it is a split parent. And usually only one of them can match, as after restart, the region
2368    // state will be changed from SPLIT to CLOSED.
2369    regionStateStore.splitRegion(parent, daughterA, daughterB, serverName, td);
2370    regionInTransitionTracker.handleRegionStateNodeOperation(node);
2371    regionInTransitionTracker.handleRegionStateNodeOperation(nodeA);
2372    regionInTransitionTracker.handleRegionStateNodeOperation(nodeB);
2373    if (shouldAssignFavoredNodes(parent)) {
2374      List<ServerName> onlineServers = this.master.getServerManager().getOnlineServersList();
2375      getFavoredNodePromoter().generateFavoredNodesForDaughter(onlineServers, parent, daughterA,
2376        daughterB);
2377    }
2378  }
2379
2380  /**
2381   * When called here, the merge has happened. The merged regions have been unassigned and the above
2382   * markRegionClosed has been called on each so they have been disassociated from a hosting Server.
2383   * The merged region will be open after this call. The merged regions are removed from hbase:meta
2384   * below. Later they are deleted from the filesystem by the catalog janitor running against
2385   * hbase:meta. It notices when the merged region no longer holds references to the old regions
2386   * (References are deleted after a compaction rewrites what the Reference points at but not until
2387   * the archiver chore runs, are the References removed).
2388   */
2389  public void markRegionAsMerged(final RegionInfo child, final ServerName serverName,
2390    RegionInfo[] mergeParents) throws IOException {
2391    final RegionStateNode node = regionStates.getOrCreateRegionStateNode(child);
2392    for (RegionInfo ri : mergeParents) {
2393      regionStates.deleteRegion(ri);
2394      regionInTransitionTracker.handleRegionDelete(ri);
2395    }
2396
2397    TableDescriptor td = master.getTableDescriptors().get(child.getTable());
2398    regionStateStore.mergeRegions(child, mergeParents, serverName, td);
2399    regionInTransitionTracker.handleRegionStateNodeOperation(node);
2400    if (shouldAssignFavoredNodes(child)) {
2401      getFavoredNodePromoter().generateFavoredNodesForMergedRegion(child, mergeParents);
2402    }
2403  }
2404
2405  /*
2406   * Favored nodes should be applied only when FavoredNodes balancer is configured and the region
2407   * belongs to a non-system table.
2408   */
2409  private boolean shouldAssignFavoredNodes(RegionInfo region) {
2410    return this.shouldAssignRegionsWithFavoredNodes
2411      && FavoredNodesManager.isFavoredNodeApplicable(region);
2412  }
2413
2414  // ============================================================================================
2415  // Assign Queue (Assign/Balance)
2416  // ============================================================================================
2417  private final ArrayList<RegionStateNode> pendingAssignQueue = new ArrayList<RegionStateNode>();
2418  private final ReentrantLock assignQueueLock = new ReentrantLock();
2419  private final Condition assignQueueFullCond = assignQueueLock.newCondition();
2420
2421  /**
2422   * Add the assign operation to the assignment queue. The pending assignment operation will be
2423   * processed, and each region will be assigned by a server using the balancer.
2424   */
2425  protected void queueAssign(final RegionStateNode regionNode) {
2426    regionNode.getProcedureEvent().suspend();
2427
2428    // TODO: quick-start for meta and the other sys-tables?
2429    assignQueueLock.lock();
2430    try {
2431      pendingAssignQueue.add(regionNode);
2432      if (
2433        regionNode.isSystemTable() || pendingAssignQueue.size() == 1
2434          || pendingAssignQueue.size() >= assignDispatchWaitQueueMaxSize
2435      ) {
2436        assignQueueFullCond.signal();
2437      }
2438    } finally {
2439      assignQueueLock.unlock();
2440    }
2441  }
2442
2443  private void startAssignmentThread() {
2444    assignThread = new Thread(master.getServerName().toShortString()) {
2445      @Override
2446      public void run() {
2447        while (isRunning()) {
2448          processAssignQueue();
2449        }
2450        pendingAssignQueue.clear();
2451      }
2452    };
2453    assignThread.setDaemon(true);
2454    assignThread.start();
2455  }
2456
2457  private void stopAssignmentThread() {
2458    assignQueueSignal();
2459    try {
2460      while (assignThread.isAlive()) {
2461        assignQueueSignal();
2462        assignThread.join(250);
2463      }
2464    } catch (InterruptedException e) {
2465      LOG.warn("join interrupted", e);
2466      Thread.currentThread().interrupt();
2467    }
2468  }
2469
2470  private void assignQueueSignal() {
2471    assignQueueLock.lock();
2472    try {
2473      assignQueueFullCond.signal();
2474    } finally {
2475      assignQueueLock.unlock();
2476    }
2477  }
2478
2479  @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
2480  private HashMap<RegionInfo, RegionStateNode> waitOnAssignQueue() {
2481    HashMap<RegionInfo, RegionStateNode> regions = null;
2482
2483    assignQueueLock.lock();
2484    try {
2485      if (pendingAssignQueue.isEmpty() && isRunning()) {
2486        assignQueueFullCond.await();
2487      }
2488
2489      if (!isRunning()) {
2490        return null;
2491      }
2492      assignQueueFullCond.await(assignDispatchWaitMillis, TimeUnit.MILLISECONDS);
2493      regions = new HashMap<RegionInfo, RegionStateNode>(pendingAssignQueue.size());
2494      for (RegionStateNode regionNode : pendingAssignQueue) {
2495        regions.put(regionNode.getRegionInfo(), regionNode);
2496      }
2497      pendingAssignQueue.clear();
2498    } catch (InterruptedException e) {
2499      LOG.warn("got interrupted ", e);
2500      Thread.currentThread().interrupt();
2501    } finally {
2502      assignQueueLock.unlock();
2503    }
2504    return regions;
2505  }
2506
2507  private void processAssignQueue() {
2508    final HashMap<RegionInfo, RegionStateNode> regions = waitOnAssignQueue();
2509    if (regions == null || regions.size() == 0 || !isRunning()) {
2510      return;
2511    }
2512
2513    if (LOG.isTraceEnabled()) {
2514      LOG.trace("PROCESS ASSIGN QUEUE regionCount=" + regions.size());
2515    }
2516
2517    // TODO: Optimize balancer. pass a RegionPlan?
2518    final HashMap<RegionInfo, ServerName> retainMap = new HashMap<>();
2519    final List<RegionInfo> userHRIs = new ArrayList<>(regions.size());
2520    // Regions for system tables requiring reassignment
2521    final List<RegionInfo> systemHRIs = new ArrayList<>();
2522    for (RegionStateNode regionStateNode : regions.values()) {
2523      boolean sysTable = regionStateNode.isSystemTable();
2524      final List<RegionInfo> hris = sysTable ? systemHRIs : userHRIs;
2525      if (regionStateNode.getRegionLocation() != null) {
2526        retainMap.put(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation());
2527      } else {
2528        hris.add(regionStateNode.getRegionInfo());
2529      }
2530    }
2531
2532    // TODO: connect with the listener to invalidate the cache
2533
2534    // TODO use events
2535    List<ServerName> servers = master.getServerManager().createDestinationServersList();
2536    for (int i = 0; servers.size() < 1; ++i) {
2537      // Report every fourth time around this loop; try not to flood log.
2538      if (i % 4 == 0) {
2539        LOG.warn("No servers available; cannot place " + regions.size() + " unassigned regions.");
2540      }
2541
2542      if (!isRunning()) {
2543        LOG.debug("Stopped! Dropping assign of " + regions.size() + " queued regions.");
2544        return;
2545      }
2546      Threads.sleep(250);
2547      servers = master.getServerManager().createDestinationServersList();
2548    }
2549
2550    if (!systemHRIs.isEmpty()) {
2551      // System table regions requiring reassignment are present, get region servers
2552      // not available for system table regions
2553      final List<ServerName> excludeServers = getExcludedServersForSystemTable();
2554      List<ServerName> serversForSysTables =
2555        servers.stream().filter(s -> !excludeServers.contains(s)).collect(Collectors.toList());
2556      if (serversForSysTables.isEmpty()) {
2557        LOG.warn("Filtering old server versions and the excluded produced an empty set; "
2558          + "instead considering all candidate servers!");
2559      }
2560      LOG.debug("Processing assignQueue; systemServersCount=" + serversForSysTables.size()
2561        + ", allServersCount=" + servers.size());
2562      processAssignmentPlans(regions, null, systemHRIs,
2563        serversForSysTables.isEmpty() && !containsBogusAssignments(regions, systemHRIs)
2564          ? servers
2565          : serversForSysTables);
2566    }
2567
2568    processAssignmentPlans(regions, retainMap, userHRIs, servers);
2569  }
2570
2571  private boolean containsBogusAssignments(Map<RegionInfo, RegionStateNode> regions,
2572    List<RegionInfo> hirs) {
2573    for (RegionInfo ri : hirs) {
2574      if (
2575        regions.get(ri).getRegionLocation() != null
2576          && regions.get(ri).getRegionLocation().equals(LoadBalancer.BOGUS_SERVER_NAME)
2577      ) {
2578        return true;
2579      }
2580    }
2581    return false;
2582  }
2583
2584  private void processAssignmentPlans(final HashMap<RegionInfo, RegionStateNode> regions,
2585    final HashMap<RegionInfo, ServerName> retainMap, final List<RegionInfo> hris,
2586    final List<ServerName> servers) {
2587    boolean isTraceEnabled = LOG.isTraceEnabled();
2588    if (isTraceEnabled) {
2589      LOG.trace("Available servers count=" + servers.size() + ": " + servers);
2590    }
2591
2592    final LoadBalancer balancer = getBalancer();
2593    // ask the balancer where to place regions
2594    if (retainMap != null && !retainMap.isEmpty()) {
2595      if (isTraceEnabled) {
2596        LOG.trace("retain assign regions=" + retainMap);
2597      }
2598      try {
2599        acceptPlan(regions, balancer.retainAssignment(retainMap, servers));
2600      } catch (IOException e) {
2601        LOG.warn("unable to retain assignment", e);
2602        addToPendingAssignment(regions, retainMap.keySet());
2603      }
2604    }
2605
2606    // TODO: Do we need to split retain and round-robin?
2607    // the retain seems to fallback to round-robin/random if the region is not in the map.
2608    if (!hris.isEmpty()) {
2609      Collections.sort(hris, RegionInfo.COMPARATOR);
2610      if (isTraceEnabled) {
2611        LOG.trace("round robin regions=" + hris);
2612      }
2613      try {
2614        acceptPlan(regions, balancer.roundRobinAssignment(hris, servers));
2615      } catch (IOException e) {
2616        LOG.warn("unable to round-robin assignment", e);
2617        addToPendingAssignment(regions, hris);
2618      }
2619    }
2620  }
2621
2622  private void acceptPlan(final HashMap<RegionInfo, RegionStateNode> regions,
2623    final Map<ServerName, List<RegionInfo>> plan) throws HBaseIOException {
2624    final ProcedureEvent<?>[] events = new ProcedureEvent[regions.size()];
2625    final long st = EnvironmentEdgeManager.currentTime();
2626
2627    if (plan.isEmpty()) {
2628      throw new HBaseIOException("unable to compute plans for regions=" + regions.size());
2629    }
2630
2631    int evcount = 0;
2632    for (Map.Entry<ServerName, List<RegionInfo>> entry : plan.entrySet()) {
2633      final ServerName server = entry.getKey();
2634      for (RegionInfo hri : entry.getValue()) {
2635        final RegionStateNode regionNode = regions.get(hri);
2636        regionNode.setRegionLocation(server);
2637        if (server.equals(LoadBalancer.BOGUS_SERVER_NAME) && regionNode.isSystemTable()) {
2638          assignQueueLock.lock();
2639          try {
2640            pendingAssignQueue.add(regionNode);
2641          } finally {
2642            assignQueueLock.unlock();
2643          }
2644        } else {
2645          events[evcount++] = regionNode.getProcedureEvent();
2646        }
2647      }
2648    }
2649    ProcedureEvent.wakeEvents(getProcedureScheduler(), events);
2650
2651    final long et = EnvironmentEdgeManager.currentTime();
2652    if (LOG.isTraceEnabled()) {
2653      LOG.trace("ASSIGN ACCEPT " + events.length + " -> " + StringUtils.humanTimeDiff(et - st));
2654    }
2655  }
2656
2657  private void addToPendingAssignment(final HashMap<RegionInfo, RegionStateNode> regions,
2658    final Collection<RegionInfo> pendingRegions) {
2659    assignQueueLock.lock();
2660    try {
2661      for (RegionInfo hri : pendingRegions) {
2662        pendingAssignQueue.add(regions.get(hri));
2663      }
2664    } finally {
2665      assignQueueLock.unlock();
2666    }
2667  }
2668
2669  /**
2670   * For a given cluster with mixed versions of servers, get a list of servers with lower versions,
2671   * where system table regions should not be assigned to. For system table, we must assign regions
2672   * to a server with highest version. However, we can disable this exclusion using config:
2673   * "hbase.min.version.move.system.tables" if checkForMinVersion is true. Detailed explanation
2674   * available with definition of minVersionToMoveSysTables.
2675   * @return List of Excluded servers for System table regions.
2676   */
2677  public List<ServerName> getExcludedServersForSystemTable() {
2678    // TODO: This should be a cached list kept by the ServerManager rather than calculated on each
2679    // move or system region assign. The RegionServerTracker keeps list of online Servers with
2680    // RegionServerInfo that includes Version.
2681    List<Pair<ServerName, String>> serverList =
2682      master.getServerManager().getOnlineServersList().stream()
2683        .map(s -> new Pair<>(s, master.getRegionServerVersion(s))).collect(Collectors.toList());
2684    if (serverList.isEmpty()) {
2685      return new ArrayList<>();
2686    }
2687    String highestVersion = Collections
2688      .max(serverList, (o1, o2) -> VersionInfo.compareVersion(o1.getSecond(), o2.getSecond()))
2689      .getSecond();
2690    if (!DEFAULT_MIN_VERSION_MOVE_SYS_TABLES_CONFIG.equals(minVersionToMoveSysTables)) {
2691      int comparedValue = VersionInfo.compareVersion(minVersionToMoveSysTables, highestVersion);
2692      if (comparedValue > 0) {
2693        return new ArrayList<>();
2694      }
2695    }
2696    return serverList.stream().filter(pair -> !pair.getSecond().equals(highestVersion))
2697      .map(Pair::getFirst).collect(Collectors.toList());
2698  }
2699
2700  MasterServices getMaster() {
2701    return master;
2702  }
2703
2704  /** Returns a snapshot of rsReports */
2705  public Map<ServerName, Set<byte[]>> getRSReports() {
2706    Map<ServerName, Set<byte[]>> rsReportsSnapshot = new HashMap<>();
2707    synchronized (rsReports) {
2708      rsReports.entrySet().forEach(e -> rsReportsSnapshot.put(e.getKey(), e.getValue()));
2709    }
2710    return rsReportsSnapshot;
2711  }
2712
2713  /**
2714   * Provide regions state count for given table. e.g howmany regions of give table are
2715   * opened/closed/rit etc
2716   * @param tableName TableName
2717   * @return region states count
2718   */
2719  public RegionStatesCount getRegionStatesCount(TableName tableName) {
2720    int openRegionsCount = 0;
2721    int closedRegionCount = 0;
2722    int ritCount = 0;
2723    int splitRegionCount = 0;
2724    int totalRegionCount = 0;
2725    if (!isTableDisabled(tableName)) {
2726      final List<RegionState> states = regionStates.getTableRegionStates(tableName);
2727      for (RegionState regionState : states) {
2728        if (regionState.isOpened()) {
2729          openRegionsCount++;
2730        } else if (regionState.isClosed()) {
2731          closedRegionCount++;
2732        } else if (regionState.isSplit()) {
2733          splitRegionCount++;
2734        }
2735      }
2736      totalRegionCount = states.size();
2737      ritCount = totalRegionCount - openRegionsCount - splitRegionCount;
2738    }
2739    return new RegionStatesCount.RegionStatesCountBuilder().setOpenRegions(openRegionsCount)
2740      .setClosedRegions(closedRegionCount).setSplitRegions(splitRegionCount)
2741      .setRegionsInTransition(ritCount).setTotalRegions(totalRegionCount).build();
2742  }
2743
2744}