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