2525
2626import java .io .IOException ;
2727import java .util .List ;
28+ import java .util .concurrent .CompletableFuture ;
2829import java .util .concurrent .ExecutorService ;
2930import java .util .concurrent .Executors ;
30- import java .util .concurrent . Future ;
31+ import java .util .function . LongConsumer ;
3132import java .util .stream .Collectors ;
33+ import org .apache .hadoop .conf .Configuration ;
3234import org .apache .hadoop .hbase .master .procedure .GlobalProcedureInterface ;
3335import org .apache .hadoop .hbase .master .procedure .MasterProcedureEnv ;
3436import org .apache .hadoop .hbase .master .procedure .PeerProcedureInterface ;
3537import org .apache .hadoop .hbase .procedure2 .ProcedureStateSerializer ;
3638import org .apache .hadoop .hbase .procedure2 .ProcedureSuspendedException ;
39+ import org .apache .hadoop .hbase .procedure2 .ProcedureUtil ;
3740import org .apache .hadoop .hbase .procedure2 .ProcedureYieldException ;
3841import org .apache .hadoop .hbase .procedure2 .StateMachineProcedure ;
3942import org .apache .hadoop .hbase .replication .ReplicationPeerDescription ;
4043import org .apache .hadoop .hbase .replication .ZKReplicationQueueStorageForMigration ;
44+ import org .apache .hadoop .hbase .util .FutureUtils ;
45+ import org .apache .hadoop .hbase .util .IdLock ;
46+ import org .apache .hadoop .hbase .util .RetryCounter ;
4147import org .apache .hadoop .hbase .util .VersionInfo ;
4248import org .apache .yetus .audience .InterfaceAudience ;
4349import org .apache .zookeeper .KeeperException ;
@@ -65,18 +71,34 @@ public class MigrateReplicationQueueFromZkToTableProcedure
6571
6672 private List <String > disabledPeerIds ;
6773
68- private List < Future <?>> futures ;
74+ private CompletableFuture <?> future ;
6975
7076 private ExecutorService executor ;
7177
78+ private RetryCounter retryCounter ;
79+
7280 @ Override
7381 public String getGlobalId () {
7482 return getClass ().getSimpleName ();
7583 }
7684
85+ private ProcedureSuspendedException suspend (Configuration conf , LongConsumer backoffConsumer )
86+ throws ProcedureSuspendedException {
87+ if (retryCounter == null ) {
88+ retryCounter = ProcedureUtil .createRetryCounter (conf );
89+ }
90+ long backoff = retryCounter .getBackoffTimeAndIncrementAttempts ();
91+ backoffConsumer .accept (backoff );
92+ throw suspend (Math .toIntExact (backoff ), true );
93+ }
94+
95+ private void resetRetry () {
96+ retryCounter = null ;
97+ }
98+
7799 private ExecutorService getExecutorService () {
78100 if (executor == null ) {
79- executor = Executors .newFixedThreadPool ( 3 , new ThreadFactoryBuilder ()
101+ executor = Executors .newCachedThreadPool ( new ThreadFactoryBuilder ()
80102 .setNameFormat (getClass ().getSimpleName () + "-%d" ).setDaemon (true ).build ());
81103 }
82104 return executor ;
@@ -95,14 +117,17 @@ private void waitUntilNoPeerProcedure(MasterProcedureEnv env) throws ProcedureSu
95117 peerProcCount = env .getMasterServices ().getProcedures ().stream ()
96118 .filter (p -> p instanceof PeerProcedureInterface ).filter (p -> !p .isFinished ()).count ();
97119 } catch (IOException e ) {
98- LOG .warn ("failed to check peer procedure status" , e );
99- throw suspend (5000 , true );
120+ throw suspend (env .getMasterConfiguration (),
121+ backoff -> LOG .warn ("failed to check peer procedure status, sleep {} secs and retry later" ,
122+ backoff / 1000 , e ));
100123 }
101124 if (peerProcCount > 0 ) {
102- LOG .info ("There are still {} pending peer procedures, will sleep and check later" ,
103- peerProcCount );
104- throw suspend (10_000 , true );
125+ throw suspend (env .getMasterConfiguration (),
126+ backoff -> LOG .info (
127+ "There are still {} pending peer procedures, sleep {} secs and retry later" ,
128+ peerProcCount , backoff / 1000 ));
105129 }
130+ resetRetry ();
106131 LOG .info ("No pending peer procedures found, continue..." );
107132 }
108133
@@ -122,8 +147,10 @@ protected Flow executeFromState(MasterProcedureEnv env,
122147 try {
123148 oldStorage .deleteAllData ();
124149 } catch (KeeperException e ) {
125- LOG .warn ("failed to delete old replication queue data, sleep and retry later" , e );
126- suspend (10_000 , true );
150+ throw suspend (env .getMasterConfiguration (),
151+ backoff -> LOG .warn (
152+ "failed to delete old replication queue data, sleep {} secs and retry later" ,
153+ backoff / 1000 , e ));
127154 }
128155 return Flow .NO_MORE_STATE ;
129156 }
@@ -132,6 +159,7 @@ protected Flow executeFromState(MasterProcedureEnv env,
132159 disabledPeerIds = peers .stream ().filter (ReplicationPeerDescription ::isEnabled )
133160 .map (ReplicationPeerDescription ::getPeerId ).collect (Collectors .toList ());
134161 setNextState (MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER );
162+ resetRetry ();
135163 return Flow .HAS_MORE_STATE ;
136164 case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER :
137165 for (String peerId : disabledPeerIds ) {
@@ -140,39 +168,52 @@ protected Flow executeFromState(MasterProcedureEnv env,
140168 setNextState (MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE );
141169 return Flow .HAS_MORE_STATE ;
142170 case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE :
143- if (futures != null ) {
144- // wait until all futures done
145- long notDone = futures .stream ().filter (f -> !f .isDone ()).count ();
146- if (notDone == 0 ) {
147- boolean succ = true ;
148- for (Future <?> future : futures ) {
149- try {
150- future .get ();
151- } catch (Exception e ) {
152- succ = false ;
153- LOG .warn ("Failed to migrate" , e );
154- }
155- }
156- if (succ ) {
157- shutdownExecutorService ();
158- setNextState (MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING );
159- return Flow .HAS_MORE_STATE ;
160- }
161- // reschedule to retry migration again
162- futures = null ;
163- } else {
164- LOG .info ("There still {} pending migration tasks, will sleep and check later" , notDone );
165- throw suspend (10_000 , true );
171+ if (future != null ) {
172+ // should have finished when we arrive here
173+ assert future .isDone ();
174+ try {
175+ future .get ();
176+ } catch (Exception e ) {
177+ future = null ;
178+ throw suspend (env .getMasterConfiguration (),
179+ backoff -> LOG .warn ("failed to migrate queue data, sleep {} secs and retry later" ,
180+ backoff / 1000 , e ));
166181 }
182+ shutdownExecutorService ();
183+ setNextState (MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING );
184+ resetRetry ();
185+ return Flow .HAS_MORE_STATE ;
167186 }
168- try {
169- futures = env .getReplicationPeerManager ()
170- .migrateQueuesFromZk (env .getMasterServices ().getZooKeeper (), getExecutorService ());
171- } catch (IOException e ) {
172- LOG .warn ("failed to submit migration tasks" , e );
173- throw suspend (10_000 , true );
174- }
175- throw suspend (10_000 , true );
187+ future = env .getReplicationPeerManager ()
188+ .migrateQueuesFromZk (env .getMasterServices ().getZooKeeper (), getExecutorService ());
189+ FutureUtils .addListener (future , (r , e ) -> {
190+ // should acquire procedure execution lock to make sure that the procedure executor has
191+ // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be
192+ // race and cause unexpected result
193+ IdLock procLock =
194+ env .getMasterServices ().getMasterProcedureExecutor ().getProcExecutionLock ();
195+ IdLock .Entry lockEntry ;
196+ try {
197+ lockEntry = procLock .getLockEntry (getProcId ());
198+ } catch (IOException ioe ) {
199+ LOG .error ("Error while acquiring execution lock for procedure {}"
200+ + " when trying to wake it up, aborting..." , ioe );
201+ env .getMasterServices ().abort ("Can not acquire procedure execution lock" , e );
202+ return ;
203+ }
204+ try {
205+ setTimeoutFailure (env );
206+ } finally {
207+ procLock .releaseLockEntry (lockEntry );
208+ }
209+ });
210+ // here we set timeout to -1 so the ProcedureExecutor will not schedule a Timer for us
211+ setTimeout (-1 );
212+ setState (ProcedureProtos .ProcedureState .WAITING_TIMEOUT );
213+ // skip persistence is a must now since when restarting, if the procedure is in
214+ // WAITING_TIMEOUT state and has -1 as timeout, it will block there forever...
215+ skipPersistence ();
216+ throw new ProcedureSuspendedException ();
176217 case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING :
177218 long rsWithLowerVersion =
178219 env .getMasterServices ().getServerManager ().getOnlineServers ().values ().stream ()
@@ -181,9 +222,11 @@ protected Flow executeFromState(MasterProcedureEnv env,
181222 setNextState (MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER );
182223 return Flow .HAS_MORE_STATE ;
183224 } else {
184- LOG .info ("There are still {} region servers which have a major version less than {}, "
185- + "will sleep and check later" , rsWithLowerVersion , MIN_MAJOR_VERSION );
186- throw suspend (10_000 , true );
225+ throw suspend (env .getMasterConfiguration (),
226+ backoff -> LOG .warn (
227+ "There are still {} region servers which have a major version"
228+ + " less than {}, sleep {} secs and check later" ,
229+ rsWithLowerVersion , MIN_MAJOR_VERSION , backoff / 1000 ));
187230 }
188231 case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER :
189232 for (String peerId : disabledPeerIds ) {
0 commit comments