Skip to content

Commit ff31d25

Browse files
committed
HBASE-27430 Should disable replication log cleaner when migrating replication queue data (#4901)
Signed-off-by: Liangjun He <heliangjun@apache.org>
1 parent ac878a5 commit ff31d25

File tree

3 files changed

+80
-8
lines changed

3 files changed

+80
-8
lines changed

hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -724,11 +724,13 @@ message AssignReplicationQueuesStateData {
724724
}
725725

726726
enum MigrateReplicationQueueFromZkToTableState {
727-
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE = 1;
728-
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER = 2;
729-
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE = 3;
730-
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING = 4;
731-
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER = 5;
727+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER = 1;
728+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE = 2;
729+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER = 3;
730+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE = 4;
731+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING = 5;
732+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER = 6;
733+
MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER = 7;
732734
}
733735

734736
message MigrateReplicationQueueFromZkToTableStateData {

hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java

Lines changed: 45 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,9 @@
1717
*/
1818
package org.apache.hadoop.hbase.master.replication;
1919

20+
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER;
2021
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER;
22+
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER;
2123
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER;
2224
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE;
2325
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE;
@@ -111,6 +113,26 @@ private void shutdownExecutorService() {
111113
}
112114
}
113115

116+
private void disableReplicationLogCleaner(MasterProcedureEnv env)
117+
throws ProcedureSuspendedException {
118+
if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
119+
// it is not likely that we can reach here as we will schedule this procedure immediately
120+
// after master restarting, where ReplicationLogCleaner should have not started its first run
121+
// yet. But anyway, let's make the code more robust. And it is safe to wait a bit here since
122+
// there will be no data in the new replication queue storage before we execute this procedure
123+
// so ReplicationLogCleaner will quit immediately without doing anything.
124+
throw suspend(env.getMasterConfiguration(),
125+
backoff -> LOG.info(
126+
"Can not disable replication log cleaner, sleep {} secs and retry later",
127+
backoff / 1000));
128+
}
129+
resetRetry();
130+
}
131+
132+
private void enableReplicationLogCleaner(MasterProcedureEnv env) {
133+
env.getReplicationPeerManager().getReplicationLogCleanerBarrier().enable();
134+
}
135+
114136
private void waitUntilNoPeerProcedure(MasterProcedureEnv env) throws ProcedureSuspendedException {
115137
long peerProcCount;
116138
try {
@@ -136,6 +158,10 @@ protected Flow executeFromState(MasterProcedureEnv env,
136158
MigrateReplicationQueueFromZkToTableState state)
137159
throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
138160
switch (state) {
161+
case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER:
162+
disableReplicationLogCleaner(env);
163+
setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE);
164+
return Flow.HAS_MORE_STATE;
139165
case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE:
140166
waitUntilNoPeerProcedure(env);
141167
List<ReplicationPeerDescription> peers = env.getReplicationPeerManager().listPeers(null);
@@ -152,7 +178,8 @@ protected Flow executeFromState(MasterProcedureEnv env,
152178
"failed to delete old replication queue data, sleep {} secs and retry later",
153179
backoff / 1000, e));
154180
}
155-
return Flow.NO_MORE_STATE;
181+
setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER);
182+
return Flow.HAS_MORE_STATE;
156183
}
157184
// here we do not care the peers which have already been disabled, as later we do not need
158185
// to enable them
@@ -232,6 +259,10 @@ protected Flow executeFromState(MasterProcedureEnv env,
232259
for (String peerId : disabledPeerIds) {
233260
addChildProcedure(new EnablePeerProcedure(peerId));
234261
}
262+
setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER);
263+
return Flow.HAS_MORE_STATE;
264+
case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER:
265+
enableReplicationLogCleaner(env);
235266
return Flow.NO_MORE_STATE;
236267
default:
237268
throw new UnsupportedOperationException("unhandled state=" + state);
@@ -263,7 +294,19 @@ protected int getStateId(MigrateReplicationQueueFromZkToTableState state) {
263294

264295
@Override
265296
protected MigrateReplicationQueueFromZkToTableState getInitialState() {
266-
return MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE;
297+
return MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER;
298+
}
299+
300+
@Override
301+
protected void afterReplay(MasterProcedureEnv env) {
302+
if (getCurrentState() == getInitialState()) {
303+
// do not need to disable log cleaner or acquire lock if we are in the initial state, later
304+
// when executing the procedure we will try to disable and acquire.
305+
return;
306+
}
307+
if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
308+
throw new IllegalStateException("can not disable log cleaner, this should not happen");
309+
}
267310
}
268311

269312
@Override

hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestMigrateReplicationQueueFromZkToTableProcedure.java

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,11 @@
1717
*/
1818
package org.apache.hadoop.hbase.master.replication;
1919

20+
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER;
2021
import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING;
22+
import static org.junit.Assert.assertEquals;
2123
import static org.junit.Assert.assertFalse;
24+
import static org.junit.Assert.assertTrue;
2225
import static org.mockito.Mockito.mock;
2326
import static org.mockito.Mockito.when;
2427

@@ -48,6 +51,7 @@
4851
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
4952
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
5053
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
54+
import org.apache.hadoop.hbase.replication.master.ReplicationLogCleanerBarrier;
5155
import org.apache.hadoop.hbase.testclassification.MasterTests;
5256
import org.apache.hadoop.hbase.testclassification.MediumTests;
5357
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -102,6 +106,8 @@ public Map<ServerName, ServerMetrics> getOnlineServers() {
102106

103107
@BeforeClass
104108
public static void setupCluster() throws Exception {
109+
// one hour, to make sure it will not run during the test
110+
UTIL.getConfiguration().setInt(HMaster.HBASE_MASTER_CLEANER_INTERVAL, 60 * 60 * 1000);
105111
UTIL.startMiniCluster(
106112
StartTestingClusterOption.builder().masterClass(HMasterForTest.class).build());
107113
}
@@ -193,8 +199,10 @@ public void testWaitUntilNoPeerProcedure() throws Exception {
193199
UTIL.waitFor(30000, () -> proc.isSuccess());
194200
}
195201

202+
// make sure we will disable replication peers while migrating
203+
// and also tests disable/enable replication log cleaner and wait for region server upgrading
196204
@Test
197-
public void testDisablePeerAndWaitUpgrading() throws Exception {
205+
public void testDisablePeerAndWaitStates() throws Exception {
198206
String peerId = "2";
199207
ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
200208
.setClusterKey(UTIL.getZkCluster().getAddress().toString() + ":/testhbase")
@@ -206,21 +214,40 @@ public void testDisablePeerAndWaitUpgrading() throws Exception {
206214
EXTRA_REGION_SERVERS
207215
.put(ServerName.valueOf("localhost", 54321, EnvironmentEdgeManager.currentTime()), metrics);
208216

217+
ReplicationLogCleanerBarrier barrier = UTIL.getHBaseCluster().getMaster()
218+
.getReplicationPeerManager().getReplicationLogCleanerBarrier();
219+
assertTrue(barrier.start());
220+
209221
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
210222

211223
MigrateReplicationQueueFromZkToTableProcedure proc =
212224
new MigrateReplicationQueueFromZkToTableProcedure();
213225
procExec.submitProcedure(proc);
226+
227+
Thread.sleep(5000);
228+
// make sure we are still waiting for replication log cleaner quit
229+
assertEquals(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER.getNumber(),
230+
proc.getCurrentStateId());
231+
barrier.stop();
232+
214233
// wait until we reach the wait upgrading state
215234
UTIL.waitFor(30000,
216235
() -> proc.getCurrentStateId()
217236
== MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING.getNumber()
218237
&& proc.getState() == ProcedureState.WAITING_TIMEOUT);
219238
// make sure the peer is disabled for migrating
220239
assertFalse(UTIL.getAdmin().isReplicationPeerEnabled(peerId));
240+
// make sure the replication log cleaner is disabled
241+
assertFalse(barrier.start());
221242

222243
// the procedure should finish successfully
223244
EXTRA_REGION_SERVERS.clear();
224245
UTIL.waitFor(30000, () -> proc.isSuccess());
246+
247+
// make sure the peer is enabled again
248+
assertTrue(UTIL.getAdmin().isReplicationPeerEnabled(peerId));
249+
// make sure the replication log cleaner is enabled again
250+
assertTrue(barrier.start());
251+
barrier.stop();
225252
}
226253
}

0 commit comments

Comments
 (0)