rmdmattingly commented on code in PR #5534:
URL: https://github.com/apache/hbase/pull/5534#discussion_r1409823980


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java:
##########
@@ -133,39 +183,72 @@ protected Flow executeFromState(MasterProcedureEnv env, 
ReopenTableRegionsState
             regionNode.unlock();
           }
           addChildProcedure(proc);
+          regionsReopened++;
         }
         
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_CONFIRM_REOPENED);
         return Flow.HAS_MORE_STATE;
       case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED:
-        regions = 
regions.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
-          .filter(l -> l != null).collect(Collectors.toList());
+        // update region lists based on what's been reopened
+        regions = filterReopened(env, regions);
+        currentRegionBatch = filterReopened(env, currentRegionBatch);
+
+        // existing batch didn't fully reopen, so try to resolve that first.
+        // since this is a retry, don't do the batch backoff
+        if (!currentRegionBatch.isEmpty()) {
+          return reopenIfSchedulable(env, currentRegionBatch, false);
+        }
+
         if (regions.isEmpty()) {
           return Flow.NO_MORE_STATE;
         }
-        if (regions.stream().anyMatch(loc -> canSchedule(env, loc))) {
-          retryCounter = null;
-          
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
-          return Flow.HAS_MORE_STATE;
-        }
-        // We can not schedule TRSP for all the regions need to reopen, wait 
for a while and retry
-        // again.
-        if (retryCounter == null) {
-          retryCounter = 
ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
-        }
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "There are still {} region(s) which need to be reopened for table {} 
are in "
-            + "OPENING state, suspend {}secs and try again later",
-          regions.size(), tableName, backoff / 1000);
-        setTimeout(Math.toIntExact(backoff));
-        setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
-        skipPersistence();
-        throw new ProcedureSuspendedException();
+
+        // current batch is finished, schedule more regions
+        return reopenIfSchedulable(env, regions, true);
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
     }
   }
 
+  private List<HRegionLocation> filterReopened(MasterProcedureEnv env,
+    List<HRegionLocation> regionsToCheck) {
+    return 
regionsToCheck.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
+      .filter(l -> l != null).collect(Collectors.toList());
+  }
+
+  private Flow reopenIfSchedulable(MasterProcedureEnv env, 
List<HRegionLocation> regionsToReopen,
+    boolean shouldBatchBackoff) throws ProcedureSuspendedException {
+    if (regionsToReopen.stream().anyMatch(loc -> canSchedule(env, loc))) {
+      retryCounter = null;
+      
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
+      reopenBatchSize = Math.min(reopenBatchSizeMax, 2 * reopenBatchSize);

Review Comment:
   That would mean that the default case of `no batch backoff` is stuck 
processing single region batches indefinitely



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java:
##########
@@ -133,39 +183,72 @@ protected Flow executeFromState(MasterProcedureEnv env, 
ReopenTableRegionsState
             regionNode.unlock();
           }
           addChildProcedure(proc);
+          regionsReopened++;
         }
         
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_CONFIRM_REOPENED);
         return Flow.HAS_MORE_STATE;
       case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED:
-        regions = 
regions.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
-          .filter(l -> l != null).collect(Collectors.toList());
+        // update region lists based on what's been reopened
+        regions = filterReopened(env, regions);
+        currentRegionBatch = filterReopened(env, currentRegionBatch);
+
+        // existing batch didn't fully reopen, so try to resolve that first.
+        // since this is a retry, don't do the batch backoff
+        if (!currentRegionBatch.isEmpty()) {
+          return reopenIfSchedulable(env, currentRegionBatch, false);
+        }
+
         if (regions.isEmpty()) {
           return Flow.NO_MORE_STATE;
         }
-        if (regions.stream().anyMatch(loc -> canSchedule(env, loc))) {
-          retryCounter = null;
-          
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
-          return Flow.HAS_MORE_STATE;
-        }
-        // We can not schedule TRSP for all the regions need to reopen, wait 
for a while and retry
-        // again.
-        if (retryCounter == null) {
-          retryCounter = 
ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
-        }
-        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-        LOG.info(
-          "There are still {} region(s) which need to be reopened for table {} 
are in "
-            + "OPENING state, suspend {}secs and try again later",
-          regions.size(), tableName, backoff / 1000);
-        setTimeout(Math.toIntExact(backoff));
-        setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
-        skipPersistence();
-        throw new ProcedureSuspendedException();
+
+        // current batch is finished, schedule more regions
+        return reopenIfSchedulable(env, regions, true);
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
     }
   }
 
+  private List<HRegionLocation> filterReopened(MasterProcedureEnv env,
+    List<HRegionLocation> regionsToCheck) {
+    return 
regionsToCheck.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
+      .filter(l -> l != null).collect(Collectors.toList());
+  }
+
+  private Flow reopenIfSchedulable(MasterProcedureEnv env, 
List<HRegionLocation> regionsToReopen,
+    boolean shouldBatchBackoff) throws ProcedureSuspendedException {
+    if (regionsToReopen.stream().anyMatch(loc -> canSchedule(env, loc))) {
+      retryCounter = null;
+      
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
+      reopenBatchSize = Math.min(reopenBatchSizeMax, 2 * reopenBatchSize);

Review Comment:
   That would mean that the default case of `reopenBatchBackoffMillis = 0` is 
stuck processing single region batches indefinitely



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to