diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java index 9467512fc66c..38f40b248262 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java @@ -75,6 +75,10 @@ synchronized void putIfAbsent(ServerName sn) { this.deadServers.putIfAbsent(sn, EnvironmentEdgeManager.currentTime()); } + synchronized void putIfAbsent(ServerName sn, Long crashedTime) { + this.deadServers.putIfAbsent(sn, crashedTime); + } + public synchronized int size() { return deadServers.size(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index f8abca44e4c5..5f4aeee73ce9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1033,6 +1033,9 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE Map, List>> procsByType = procedureExecutor .getActiveProceduresNoCopy().stream().collect(Collectors.groupingBy(p -> p.getClass())); + // This manager must be accessed AFTER hbase:meta is confirmed on line.. and must be initialised + // before assignment manager + this.tableStateManager = new TableStateManager(this); // Create Assignment Manager this.assignmentManager = createAssignmentManager(this, masterRegion); this.assignmentManager.start(); @@ -1053,11 +1056,10 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE // TODO: Generate the splitting and live Set in one pass instead of two as we currently do. this.regionServerTracker.upgrade( procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream() - .map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()), + .map(p -> (ServerCrashProcedure) p).collect( + Collectors.toMap(ServerCrashProcedure::getServerName, Procedure::getSubmittedTime)), Sets.union(rsListStorage.getAll(), walManager.getLiveServersFromWALDir()), walManager.getSplittingServersFromWALDir()); - // This manager must be accessed AFTER hbase:meta is confirmed on line.. - this.tableStateManager = new TableStateManager(this); startupTaskGroup.addTask("Initializing ZK system trackers"); initializeZKBasedSystemTrackers(); @@ -2007,7 +2009,7 @@ private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransi // But if there are zero regions in transition, it can skip sleep to speed up. while ( !interrupted && EnvironmentEdgeManager.currentTime() < nextBalanceStartTime - && this.assignmentManager.getRegionStates().hasRegionsInTransition() + && this.assignmentManager.getRegionTransitScheduledCount() > 0 ) { try { Thread.sleep(100); @@ -2019,8 +2021,7 @@ private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransi // Throttling by max number regions in transition while ( !interrupted && maxRegionsInTransition > 0 - && this.assignmentManager.getRegionStates().getRegionsInTransitionCount() - >= maxRegionsInTransition + && this.assignmentManager.getRegionTransitScheduledCount() >= maxRegionsInTransition && EnvironmentEdgeManager.currentTime() <= cutoffTime ) { try { @@ -2099,7 +2100,7 @@ public BalanceResponse balance(BalanceRequest request) throws IOException { synchronized (this.balancer) { // Only allow one balance run at at time. - if (this.assignmentManager.hasRegionsInTransition()) { + if (this.assignmentManager.getRegionTransitScheduledCount() > 0) { List regionsInTransition = assignmentManager.getRegionsInTransition(); // if hbase:meta region is in transition, result of assignment cannot be recorded // ignore the force flag in that case @@ -2114,8 +2115,8 @@ public BalanceResponse balance(BalanceRequest request) throws IOException { if (!request.isIgnoreRegionsInTransition() || metaInTransition) { LOG.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition - + ") because " + regionsInTransition.size() + " region(s) in transition: " + toPrint - + (truncated ? "(truncated list)" : "")); + + ") because " + assignmentManager.getRegionTransitScheduledCount() + + " region(s) in transition: " + toPrint + (truncated ? "(truncated list)" : "")); return responseBuilder.build(); } } @@ -2251,7 +2252,7 @@ public boolean normalizeRegions(final NormalizeTableFilterParams ntfp, if (skipRegionManagementAction("region normalizer")) { return false; } - if (assignmentManager.hasRegionsInTransition()) { + if (assignmentManager.getRegionTransitScheduledCount() > 0) { return false; } @@ -3081,7 +3082,7 @@ public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet