1818package org .apache .hadoop .hbase .master .replication ;
1919
2020import java .io .IOException ;
21+ import java .io .InterruptedIOException ;
2122import java .util .HashMap ;
2223import java .util .Map ;
2324import org .apache .hadoop .hbase .MetaTableAccessor ;
2425import org .apache .hadoop .hbase .TableName ;
2526import org .apache .hadoop .hbase .client .Connection ;
26- import org .apache .hadoop .hbase .client .RegionInfo ;
2727import org .apache .hadoop .hbase .client .TableDescriptor ;
28- import org .apache .hadoop .hbase .master . MasterFileSystem ;
28+ import org .apache .hadoop .hbase .client . TableState ;
2929import org .apache .hadoop .hbase .master .TableStateManager ;
3030import org .apache .hadoop .hbase .master .TableStateManager .TableStateNotFoundException ;
31- import org .apache .hadoop .hbase .master .assignment .RegionStates ;
3231import org .apache .hadoop .hbase .master .procedure .MasterProcedureEnv ;
3332import org .apache .hadoop .hbase .master .procedure .ProcedurePrepareLatch ;
3433import org .apache .hadoop .hbase .procedure2 .ProcedureSuspendedException ;
3837import org .apache .hadoop .hbase .replication .ReplicationQueueStorage ;
3938import org .apache .hadoop .hbase .replication .ReplicationUtils ;
4039import org .apache .hadoop .hbase .util .Pair ;
41- import org .apache .hadoop .hbase .wal .WALSplitter ;
4240import org .apache .yetus .audience .InterfaceAudience ;
4341import org .slf4j .Logger ;
4442import org .slf4j .LoggerFactory ;
@@ -56,6 +54,9 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
5654
5755 protected static final int UPDATE_LAST_SEQ_ID_BATCH_SIZE = 1000 ;
5856
57+ // The sleep interval when waiting table to be enabled or disabled.
58+ protected static final int SLEEP_INTERVAL_MS = 1000 ;
59+
5960 protected ModifyPeerProcedure () {
6061 }
6162
@@ -126,6 +127,27 @@ protected void updateLastPushedSequenceIdForSerialPeer(MasterProcedureEnv env)
126127 throw new UnsupportedOperationException ();
127128 }
128129
130+ // If the table is in enabling state, we need to wait until it is enabled and then reopen all its
131+ // regions.
132+ private boolean needReopen (TableStateManager tsm , TableName tn ) throws IOException {
133+ for (;;) {
134+ try {
135+ TableState state = tsm .getTableState (tn );
136+ if (state .isEnabled ()) {
137+ return true ;
138+ }
139+ if (!state .isEnabling ()) {
140+ return false ;
141+ }
142+ Thread .sleep (SLEEP_INTERVAL_MS );
143+ } catch (TableStateNotFoundException e ) {
144+ return false ;
145+ } catch (InterruptedException e ) {
146+ throw (IOException ) new InterruptedIOException (e .getMessage ()).initCause (e );
147+ }
148+ }
149+ }
150+
129151 private void reopenRegions (MasterProcedureEnv env ) throws IOException {
130152 ReplicationPeerConfig peerConfig = getNewPeerConfig ();
131153 ReplicationPeerConfig oldPeerConfig = getOldPeerConfig ();
@@ -142,15 +164,10 @@ private void reopenRegions(MasterProcedureEnv env) throws IOException {
142164 ReplicationUtils .contains (oldPeerConfig , tn )) {
143165 continue ;
144166 }
145- try {
146- if (!tsm .getTableState (tn ).isEnabled ()) {
147- continue ;
148- }
149- } catch (TableStateNotFoundException e ) {
150- continue ;
167+ if (needReopen (tsm , tn )) {
168+ addChildProcedure (env .getAssignmentManager ().createReopenProcedures (
169+ env .getAssignmentManager ().getRegionStates ().getRegionsOfTable (tn )));
151170 }
152- addChildProcedure (env .getAssignmentManager ().createReopenProcedures (
153- env .getAssignmentManager ().getRegionStates ().getRegionsOfTable (tn )));
154171 }
155172 }
156173
@@ -183,6 +200,26 @@ protected final void setLastPushedSequenceId(MasterProcedureEnv env,
183200 }
184201 }
185202
203+ // If the table is currently disabling, then we need to wait until it is disabled.We will write
204+ // replication barrier for a disabled table. And return whether we need to update the last pushed
205+ // sequence id, if the table has been deleted already, i.e, we hit TableStateNotFoundException,
206+ // then we do not need to update last pushed sequence id for this table.
207+ private boolean needSetLastPushedSequenceId (TableStateManager tsm , TableName tn )
208+ throws IOException {
209+ for (;;) {
210+ try {
211+ if (!tsm .getTableState (tn ).isDisabling ()) {
212+ return true ;
213+ }
214+ Thread .sleep (SLEEP_INTERVAL_MS );
215+ } catch (TableStateNotFoundException e ) {
216+ return false ;
217+ } catch (InterruptedException e ) {
218+ throw (IOException ) new InterruptedIOException (e .getMessage ()).initCause (e );
219+ }
220+ }
221+ }
222+
186223 // Will put the encodedRegionName->lastPushedSeqId pair into the map passed in, if the map is
187224 // large enough we will call queueStorage.setLastSequenceIds and clear the map. So the caller
188225 // should not forget to check whether the map is empty at last, if not you should call
@@ -192,26 +229,13 @@ protected final void setLastPushedSequenceIdForTable(MasterProcedureEnv env, Tab
192229 TableStateManager tsm = env .getMasterServices ().getTableStateManager ();
193230 ReplicationQueueStorage queueStorage = env .getReplicationPeerManager ().getQueueStorage ();
194231 Connection conn = env .getMasterServices ().getConnection ();
195- RegionStates regionStates = env .getAssignmentManager ().getRegionStates ();
196- MasterFileSystem mfs = env .getMasterServices ().getMasterFileSystem ();
197- boolean isTableEnabled ;
198- try {
199- isTableEnabled = tsm .getTableState (tableName ).isEnabled ();
200- } catch (TableStateNotFoundException e ) {
232+ if (!needSetLastPushedSequenceId (tsm , tableName )) {
201233 return ;
202234 }
203- if (isTableEnabled ) {
204- for (Pair <String , Long > name2Barrier : MetaTableAccessor
205- .getTableEncodedRegionNameAndLastBarrier (conn , tableName )) {
206- addToMap (lastSeqIds , name2Barrier .getFirst (), name2Barrier .getSecond ().longValue () - 1 ,
207- queueStorage );
208- }
209- } else {
210- for (RegionInfo region : regionStates .getRegionsOfTable (tableName , true )) {
211- long maxSequenceId =
212- WALSplitter .getMaxRegionSequenceId (mfs .getFileSystem (), mfs .getRegionDir (region ));
213- addToMap (lastSeqIds , region .getEncodedName (), maxSequenceId , queueStorage );
214- }
235+ for (Pair <String , Long > name2Barrier : MetaTableAccessor
236+ .getTableEncodedRegionNameAndLastBarrier (conn , tableName )) {
237+ addToMap (lastSeqIds , name2Barrier .getFirst (), name2Barrier .getSecond ().longValue () - 1 ,
238+ queueStorage );
215239 }
216240 }
217241
0 commit comments