1717 */
1818package org .apache .hadoop .hbase .master .procedure ;
1919
20+ import com .google .errorprone .annotations .RestrictedApi ;
2021import java .io .IOException ;
2122import java .util .ArrayList ;
2223import java .util .Collections ;
@@ -53,6 +54,17 @@ public class ReopenTableRegionsProcedure
5354
5455 private static final Logger LOG = LoggerFactory .getLogger (ReopenTableRegionsProcedure .class );
5556
57+ public static final String PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY =
58+ "hbase.reopen.table.regions.progressive.batch.backoff.ms" ;
59+ public static final long PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT = 0L ;
60+ public static final String PROGRESSIVE_BATCH_SIZE_MAX_KEY =
61+ "hbase.reopen.table.regions.progressive.batch.size.max" ;
62+ public static final int PROGRESSIVE_BATCH_SIZE_MAX_DISABLED = -1 ;
63+ private static final int PROGRESSIVE_BATCH_SIZE_MAX_DEFAULT_VALUE = Integer .MAX_VALUE ;
64+
65+ // this minimum prevents a max which would break this procedure
66+ private static final int MINIMUM_BATCH_SIZE_MAX = 1 ;
67+
5668 private TableName tableName ;
5769
5870 // Specify specific regions of a table to reopen.
@@ -61,20 +73,46 @@ public class ReopenTableRegionsProcedure
6173
6274 private List <HRegionLocation > regions = Collections .emptyList ();
6375
76+ private List <HRegionLocation > currentRegionBatch = Collections .emptyList ();
77+
6478 private RetryCounter retryCounter ;
6579
80+ private long reopenBatchBackoffMillis ;
81+ private int reopenBatchSize ;
82+ private int reopenBatchSizeMax ;
83+ private long regionsReopened = 0 ;
84+ private long batchesProcessed = 0 ;
85+
6686 public ReopenTableRegionsProcedure () {
67- regionNames = Collections . emptyList ( );
87+ this ( null );
6888 }
6989
7090 public ReopenTableRegionsProcedure (TableName tableName ) {
71- this .tableName = tableName ;
72- this .regionNames = Collections .emptyList ();
91+ this (tableName , Collections .emptyList ());
7392 }
7493
7594 public ReopenTableRegionsProcedure (final TableName tableName , final List <byte []> regionNames ) {
95+ this (tableName , regionNames , PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT ,
96+ PROGRESSIVE_BATCH_SIZE_MAX_DISABLED );
97+ }
98+
99+ public ReopenTableRegionsProcedure (final TableName tableName , long reopenBatchBackoffMillis ,
100+ int reopenBatchSizeMax ) {
101+ this (tableName , Collections .emptyList (), reopenBatchBackoffMillis , reopenBatchSizeMax );
102+ }
103+
104+ public ReopenTableRegionsProcedure (final TableName tableName , final List <byte []> regionNames ,
105+ long reopenBatchBackoffMillis , int reopenBatchSizeMax ) {
76106 this .tableName = tableName ;
77107 this .regionNames = regionNames ;
108+ this .reopenBatchBackoffMillis = reopenBatchBackoffMillis ;
109+ if (reopenBatchSizeMax == PROGRESSIVE_BATCH_SIZE_MAX_DISABLED ) {
110+ this .reopenBatchSize = Integer .MAX_VALUE ;
111+ this .reopenBatchSizeMax = Integer .MAX_VALUE ;
112+ } else {
113+ this .reopenBatchSize = 1 ;
114+ this .reopenBatchSizeMax = Math .max (reopenBatchSizeMax , MINIMUM_BATCH_SIZE_MAX );
115+ }
78116 }
79117
80118 @ Override
@@ -87,6 +125,30 @@ public TableOperationType getTableOperationType() {
87125 return TableOperationType .REGION_EDIT ;
88126 }
89127
128+ @ RestrictedApi (explanation = "Should only be called in tests" , link = "" ,
129+ allowedOnPath = ".*/src/test/.*" )
130+ public long getRegionsReopened () {
131+ return regionsReopened ;
132+ }
133+
134+ @ RestrictedApi (explanation = "Should only be called in tests" , link = "" ,
135+ allowedOnPath = ".*/src/test/.*" )
136+ public long getBatchesProcessed () {
137+ return batchesProcessed ;
138+ }
139+
140+ @ RestrictedApi (explanation = "Should only be called internally or in tests" , link = "" ,
141+ allowedOnPath = ".*(/src/test/.*|ReopenTableRegionsProcedure).java" )
142+ protected int progressBatchSize () {
143+ int previousBatchSize = reopenBatchSize ;
144+ reopenBatchSize = Math .min (reopenBatchSizeMax , 2 * reopenBatchSize );
145+ if (reopenBatchSize < previousBatchSize ) {
146+ // the batch size should never decrease. this must be overflow, so just use max
147+ reopenBatchSize = reopenBatchSizeMax ;
148+ }
149+ return reopenBatchSize ;
150+ }
151+
90152 private boolean canSchedule (MasterProcedureEnv env , HRegionLocation loc ) {
91153 if (loc .getSeqNum () < 0 ) {
92154 return false ;
@@ -114,7 +176,13 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState
114176 setNextState (ReopenTableRegionsState .REOPEN_TABLE_REGIONS_REOPEN_REGIONS );
115177 return Flow .HAS_MORE_STATE ;
116178 case REOPEN_TABLE_REGIONS_REOPEN_REGIONS :
117- for (HRegionLocation loc : regions ) {
179+ // if we didn't finish reopening the last batch yet, let's keep trying until we do.
180+ // at that point, the batch will be empty and we can generate a new batch
181+ if (!regions .isEmpty () && currentRegionBatch .isEmpty ()) {
182+ currentRegionBatch = regions .stream ().limit (reopenBatchSize ).collect (Collectors .toList ());
183+ batchesProcessed ++;
184+ }
185+ for (HRegionLocation loc : currentRegionBatch ) {
118186 RegionStateNode regionNode =
119187 env .getAssignmentManager ().getRegionStates ().getRegionStateNode (loc .getRegion ());
120188 // this possible, maybe the region has already been merged or split, see HBASE-20921
@@ -133,39 +201,72 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState
133201 regionNode .unlock ();
134202 }
135203 addChildProcedure (proc );
204+ regionsReopened ++;
136205 }
137206 setNextState (ReopenTableRegionsState .REOPEN_TABLE_REGIONS_CONFIRM_REOPENED );
138207 return Flow .HAS_MORE_STATE ;
139208 case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED :
140- regions = regions .stream ().map (env .getAssignmentManager ().getRegionStates ()::checkReopened )
141- .filter (l -> l != null ).collect (Collectors .toList ());
209+ // update region lists based on what's been reopened
210+ regions = filterReopened (env , regions );
211+ currentRegionBatch = filterReopened (env , currentRegionBatch );
212+
213+ // existing batch didn't fully reopen, so try to resolve that first.
214+ // since this is a retry, don't do the batch backoff
215+ if (!currentRegionBatch .isEmpty ()) {
216+ return reopenIfSchedulable (env , currentRegionBatch , false );
217+ }
218+
142219 if (regions .isEmpty ()) {
143220 return Flow .NO_MORE_STATE ;
144221 }
145- if (regions .stream ().anyMatch (loc -> canSchedule (env , loc ))) {
146- retryCounter = null ;
147- setNextState (ReopenTableRegionsState .REOPEN_TABLE_REGIONS_REOPEN_REGIONS );
148- return Flow .HAS_MORE_STATE ;
149- }
150- // We can not schedule TRSP for all the regions need to reopen, wait for a while and retry
151- // again.
152- if (retryCounter == null ) {
153- retryCounter = ProcedureUtil .createRetryCounter (env .getMasterConfiguration ());
154- }
155- long backoff = retryCounter .getBackoffTimeAndIncrementAttempts ();
156- LOG .info (
157- "There are still {} region(s) which need to be reopened for table {} are in "
158- + "OPENING state, suspend {}secs and try again later" ,
159- regions .size (), tableName , backoff / 1000 );
160- setTimeout (Math .toIntExact (backoff ));
161- setState (ProcedureProtos .ProcedureState .WAITING_TIMEOUT );
162- skipPersistence ();
163- throw new ProcedureSuspendedException ();
222+
223+ // current batch is finished, schedule more regions
224+ return reopenIfSchedulable (env , regions , true );
164225 default :
165226 throw new UnsupportedOperationException ("unhandled state=" + state );
166227 }
167228 }
168229
230+ private List <HRegionLocation > filterReopened (MasterProcedureEnv env ,
231+ List <HRegionLocation > regionsToCheck ) {
232+ return regionsToCheck .stream ().map (env .getAssignmentManager ().getRegionStates ()::checkReopened )
233+ .filter (l -> l != null ).collect (Collectors .toList ());
234+ }
235+
236+ private Flow reopenIfSchedulable (MasterProcedureEnv env , List <HRegionLocation > regionsToReopen ,
237+ boolean shouldBatchBackoff ) throws ProcedureSuspendedException {
238+ if (regionsToReopen .stream ().anyMatch (loc -> canSchedule (env , loc ))) {
239+ retryCounter = null ;
240+ setNextState (ReopenTableRegionsState .REOPEN_TABLE_REGIONS_REOPEN_REGIONS );
241+ if (shouldBatchBackoff && reopenBatchBackoffMillis > 0 ) {
242+ progressBatchSize ();
243+ setBackoffState (reopenBatchBackoffMillis );
244+ throw new ProcedureSuspendedException ();
245+ } else {
246+ return Flow .HAS_MORE_STATE ;
247+ }
248+ }
249+
250+ // We can not schedule TRSP for all the regions need to reopen, wait for a while and retry
251+ // again.
252+ if (retryCounter == null ) {
253+ retryCounter = ProcedureUtil .createRetryCounter (env .getMasterConfiguration ());
254+ }
255+ long backoffMillis = retryCounter .getBackoffTimeAndIncrementAttempts ();
256+ LOG .info (
257+ "There are still {} region(s) which need to be reopened for table {}. {} are in "
258+ + "OPENING state, suspend {}secs and try again later" ,
259+ regions .size (), tableName , currentRegionBatch .size (), backoffMillis / 1000 );
260+ setBackoffState (backoffMillis );
261+ throw new ProcedureSuspendedException ();
262+ }
263+
264+ private void setBackoffState (long millis ) {
265+ setTimeout (Math .toIntExact (millis ));
266+ setState (ProcedureProtos .ProcedureState .WAITING_TIMEOUT );
267+ skipPersistence ();
268+ }
269+
169270 private List <HRegionLocation >
170271 getRegionLocationsForReopen (List <HRegionLocation > tableRegionsForReopen ) {
171272
0 commit comments