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