Skip to content

Commit 4be9909

Browse files
[region migration] Remove retry of RegionMigrateService::addPeer (#14362)
* done * synchronized & snapshot name * remove test * for removePeer and resetPeerList
1 parent 88ffa48 commit 4be9909

File tree

4 files changed

+103
-159
lines changed

4 files changed

+103
-159
lines changed

iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java

Lines changed: 83 additions & 74 deletions
Original file line numberDiff line numberDiff line change
@@ -296,53 +296,55 @@ public void addRemotePeer(ConsensusGroupId groupId, Peer peer) throws ConsensusE
296296
IoTConsensusServerImpl impl =
297297
Optional.ofNullable(stateMachineMap.get(groupId))
298298
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
299-
if (impl.getConfiguration().contains(peer)) {
300-
throw new PeerAlreadyInConsensusGroupException(groupId, peer);
301-
}
302-
try {
303-
// step 1: inactive new Peer to prepare for following steps
304-
logger.info("[IoTConsensus] inactivate new peer: {}", peer);
305-
impl.inactivePeer(peer, false);
299+
synchronized (impl) {
300+
if (impl.getConfiguration().contains(peer)) {
301+
throw new PeerAlreadyInConsensusGroupException(groupId, peer);
302+
}
303+
try {
304+
// step 1: inactive new Peer to prepare for following steps
305+
logger.info("[IoTConsensus] inactivate new peer: {}", peer);
306+
impl.inactivePeer(peer, false);
306307

307-
// step 2: notify all the other Peers to build the sync connection to newPeer
308-
logger.info("[IoTConsensus] notify current peers to build sync log...");
309-
impl.notifyPeersToBuildSyncLogChannel(peer);
308+
// step 2: notify all the other Peers to build the sync connection to newPeer
309+
logger.info("[IoTConsensus] notify current peers to build sync log...");
310+
impl.notifyPeersToBuildSyncLogChannel(peer);
310311

311-
// step 3: take snapshot
312-
logger.info("[IoTConsensus] start to take snapshot...");
312+
// step 3: take snapshot
313+
logger.info("[IoTConsensus] start to take snapshot...");
313314

314-
impl.takeSnapshot();
315+
impl.takeSnapshot();
315316

316-
// step 4: transit snapshot
317-
logger.info("[IoTConsensus] start to transmit snapshot...");
318-
impl.transmitSnapshot(peer);
317+
// step 4: transit snapshot
318+
logger.info("[IoTConsensus] start to transmit snapshot...");
319+
impl.transmitSnapshot(peer);
319320

320-
// step 5: let the new peer load snapshot
321-
logger.info("[IoTConsensus] trigger new peer to load snapshot...");
322-
impl.triggerSnapshotLoad(peer);
323-
KillPoint.setKillPoint(DataNodeKillPoints.COORDINATOR_ADD_PEER_TRANSITION);
321+
// step 5: let the new peer load snapshot
322+
logger.info("[IoTConsensus] trigger new peer to load snapshot...");
323+
impl.triggerSnapshotLoad(peer);
324+
KillPoint.setKillPoint(DataNodeKillPoints.COORDINATOR_ADD_PEER_TRANSITION);
324325

325-
// step 6: active new Peer
326-
logger.info("[IoTConsensus] activate new peer...");
327-
impl.activePeer(peer);
326+
// step 6: active new Peer
327+
logger.info("[IoTConsensus] activate new peer...");
328+
impl.activePeer(peer);
329+
330+
// step 7: notify remote peer to clean up transferred snapshot
331+
logger.info("[IoTConsensus] clean up remote snapshot...");
332+
try {
333+
impl.cleanupRemoteSnapshot(peer);
334+
} catch (ConsensusGroupModifyPeerException e) {
335+
logger.warn("[IoTConsensus] failed to cleanup remote snapshot", e);
336+
}
337+
KillPoint.setKillPoint(DataNodeKillPoints.COORDINATOR_ADD_PEER_DONE);
328338

329-
// step 7: notify remote peer to clean up transferred snapshot
330-
logger.info("[IoTConsensus] clean up remote snapshot...");
331-
try {
332-
impl.cleanupRemoteSnapshot(peer);
333339
} catch (ConsensusGroupModifyPeerException e) {
334-
logger.warn("[IoTConsensus] failed to cleanup remote snapshot", e);
340+
logger.info("[IoTConsensus] add remote peer failed, automatic cleanup side effects...");
341+
// try to clean up the sync log channel
342+
impl.notifyPeersToRemoveSyncLogChannel(peer);
343+
throw new ConsensusException(e);
344+
} finally {
345+
logger.info("[IoTConsensus] clean up local snapshot...");
346+
impl.cleanupLocalSnapshot();
335347
}
336-
KillPoint.setKillPoint(DataNodeKillPoints.COORDINATOR_ADD_PEER_DONE);
337-
338-
} catch (ConsensusGroupModifyPeerException e) {
339-
logger.info("[IoTConsensus] add remote peer failed, automatic cleanup side effects...");
340-
// try to clean up the sync log channel
341-
impl.notifyPeersToRemoveSyncLogChannel(peer);
342-
throw new ConsensusException(e);
343-
} finally {
344-
logger.info("[IoTConsensus] clean up local snapshot...");
345-
impl.cleanupLocalSnapshot();
346348
}
347349
}
348350

@@ -352,29 +354,32 @@ public void removeRemotePeer(ConsensusGroupId groupId, Peer peer) throws Consens
352354
Optional.ofNullable(stateMachineMap.get(groupId))
353355
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
354356

355-
if (!impl.getConfiguration().contains(peer)) {
356-
throw new PeerNotInConsensusGroupException(groupId, peer.toString());
357-
}
357+
synchronized (impl) {
358+
if (!impl.getConfiguration().contains(peer)) {
359+
throw new PeerNotInConsensusGroupException(groupId, peer.toString());
360+
}
358361

359-
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.INIT);
362+
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.INIT);
360363

361-
// let other peers remove the sync channel with target peer
362-
impl.notifyPeersToRemoveSyncLogChannel(peer);
363-
KillPoint.setKillPoint(
364-
IoTConsensusRemovePeerCoordinatorKillPoints.AFTER_NOTIFY_PEERS_TO_REMOVE_REPLICATE_CHANNEL);
364+
// let other peers remove the sync channel with target peer
365+
impl.notifyPeersToRemoveSyncLogChannel(peer);
366+
KillPoint.setKillPoint(
367+
IoTConsensusRemovePeerCoordinatorKillPoints
368+
.AFTER_NOTIFY_PEERS_TO_REMOVE_REPLICATE_CHANNEL);
365369

366-
try {
367-
// let target peer reject new write
368-
impl.inactivePeer(peer, true);
369-
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.AFTER_INACTIVE_PEER);
370-
// wait its SyncLog to complete
371-
impl.waitTargetPeerUntilSyncLogCompleted(peer);
372-
// wait its region related resource to release
373-
impl.waitReleaseAllRegionRelatedResource(peer);
374-
} catch (ConsensusGroupModifyPeerException e) {
375-
throw new ConsensusException(e.getMessage());
370+
try {
371+
// let target peer reject new write
372+
impl.inactivePeer(peer, true);
373+
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.AFTER_INACTIVE_PEER);
374+
// wait its SyncLog to complete
375+
impl.waitTargetPeerUntilSyncLogCompleted(peer);
376+
// wait its region related resource to release
377+
impl.waitReleaseAllRegionRelatedResource(peer);
378+
} catch (ConsensusGroupModifyPeerException e) {
379+
throw new ConsensusException(e.getMessage());
380+
}
381+
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.FINISH);
376382
}
377-
KillPoint.setKillPoint(IoTConsensusRemovePeerCoordinatorKillPoints.FINISH);
378383
}
379384

380385
@Override
@@ -485,6 +490,7 @@ public void resetPeerList(ConsensusGroupId groupId, List<Peer> correctPeers)
485490
IoTConsensusServerImpl impl =
486491
Optional.ofNullable(stateMachineMap.get(groupId))
487492
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));
493+
488494
Peer localPeer = new Peer(groupId, thisNodeId, thisNode);
489495
if (!correctPeers.contains(localPeer)) {
490496
logger.info(
@@ -493,25 +499,28 @@ public void resetPeerList(ConsensusGroupId groupId, List<Peer> correctPeers)
493499
deleteLocalPeer(groupId);
494500
return;
495501
}
496-
ImmutableList<Peer> currentMembers = ImmutableList.copyOf(impl.getConfiguration());
497-
String previousPeerListStr = currentMembers.toString();
498-
for (Peer peer : currentMembers) {
499-
if (!correctPeers.contains(peer)) {
500-
if (!impl.removeSyncLogChannel(peer)) {
501-
logger.error(
502-
"[RESET PEER LIST] Failed to remove peer {}'s sync log channel from group {}",
503-
peer,
504-
groupId);
502+
503+
synchronized (impl) {
504+
ImmutableList<Peer> currentMembers = ImmutableList.copyOf(impl.getConfiguration());
505+
String previousPeerListStr = currentMembers.toString();
506+
for (Peer peer : currentMembers) {
507+
if (!correctPeers.contains(peer)) {
508+
if (!impl.removeSyncLogChannel(peer)) {
509+
logger.error(
510+
"[RESET PEER LIST] Failed to remove peer {}'s sync log channel from group {}",
511+
peer,
512+
groupId);
513+
}
505514
}
506515
}
507-
}
508-
logger.info(
509-
"[RESET PEER LIST] Local peer list has been reset: {} -> {}",
510-
previousPeerListStr,
511-
impl.getConfiguration());
512-
for (Peer peer : correctPeers) {
513-
if (!impl.getConfiguration().contains(peer)) {
514-
logger.warn("[RESET PEER LIST] \"Correct peer\" {} is not in local peer list", peer);
516+
logger.info(
517+
"[RESET PEER LIST] Local peer list has been reset: {} -> {}",
518+
previousPeerListStr,
519+
impl.getConfiguration());
520+
for (Peer peer : correctPeers) {
521+
if (!impl.getConfiguration().contains(peer)) {
522+
logger.warn("[RESET PEER LIST] \"Correct peer\" {} is not in local peer list", peer);
523+
}
515524
}
516525
}
517526
}

iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java

Lines changed: 2 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,7 @@
9191
import java.util.List;
9292
import java.util.PriorityQueue;
9393
import java.util.Set;
94+
import java.util.UUID;
9495
import java.util.concurrent.ConcurrentHashMap;
9596
import java.util.concurrent.ScheduledExecutorService;
9697
import java.util.concurrent.TimeUnit;
@@ -276,10 +277,9 @@ public DataSet read(IConsensusRequest request) {
276277

277278
public void takeSnapshot() throws ConsensusGroupModifyPeerException {
278279
try {
279-
long newSnapshotIndex = getLatestSnapshotIndex() + 1;
280280
newSnapshotDirName =
281281
String.format(
282-
"%s_%s_%d", SNAPSHOT_DIR_NAME, thisNode.getGroupId().getId(), newSnapshotIndex);
282+
"%s_%s_%s", SNAPSHOT_DIR_NAME, thisNode.getGroupId().getId(), UUID.randomUUID());
283283
File snapshotDir = new File(storageDir, newSnapshotDirName);
284284
if (snapshotDir.exists()) {
285285
FileUtils.deleteDirectory(snapshotDir);
@@ -400,22 +400,6 @@ private String calculateSnapshotPath(String snapshotId, String originalFilePath)
400400
return originalFilePath.substring(originalFilePath.indexOf(snapshotId));
401401
}
402402

403-
private long getLatestSnapshotIndex() {
404-
long snapShotIndex = 0;
405-
File directory = new File(storageDir);
406-
File[] versionFiles = directory.listFiles((dir, name) -> name.startsWith(SNAPSHOT_DIR_NAME));
407-
if (versionFiles == null || versionFiles.length == 0) {
408-
return snapShotIndex;
409-
}
410-
for (File file : versionFiles) {
411-
snapShotIndex =
412-
Math.max(
413-
snapShotIndex,
414-
Long.parseLong(SNAPSHOT_INDEX_PATTEN.matcher(file.getName()).replaceAll("")));
415-
}
416-
return snapShotIndex;
417-
}
418-
419403
private void clearOldSnapshot() {
420404
File directory = new File(storageDir);
421405
File[] versionFiles = directory.listFiles((dir, name) -> name.startsWith(SNAPSHOT_DIR_NAME));

iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/iot/StabilityTest.java

Lines changed: 0 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@
2929
import org.apache.iotdb.consensus.config.ConsensusConfig;
3030
import org.apache.iotdb.consensus.exception.ConsensusException;
3131
import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException;
32-
import org.apache.iotdb.consensus.exception.ConsensusGroupModifyPeerException;
3332
import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException;
3433
import org.apache.iotdb.consensus.exception.IllegalPeerEndpointException;
3534
import org.apache.iotdb.consensus.exception.IllegalPeerNumException;
@@ -95,7 +94,6 @@ public void allTest() throws Exception {
9594
peerTest();
9695
transferLeader();
9796
snapshotTest();
98-
snapshotUpgradeTest();
9997
}
10098

10199
public void addConsensusGroup() {
@@ -212,38 +210,4 @@ public void snapshotTest() throws ConsensusException {
212210
Assert.assertNotEquals(versionFiles1[0].getName(), versionFiles2[0].getName());
213211
consensusImpl.deleteLocalPeer(dataRegionId);
214212
}
215-
216-
public void snapshotUpgradeTest() throws Exception {
217-
consensusImpl.createLocalPeer(
218-
dataRegionId,
219-
Collections.singletonList(new Peer(dataRegionId, 1, new TEndPoint("0.0.0.0", basePort))));
220-
consensusImpl.triggerSnapshot(dataRegionId, false);
221-
long oldSnapshotIndex = System.currentTimeMillis();
222-
String oldSnapshotDirName =
223-
String.format(
224-
"%s_%s_%d",
225-
IoTConsensusServerImpl.SNAPSHOT_DIR_NAME, dataRegionId.getId(), oldSnapshotIndex);
226-
File regionDir = new File(storageDir, "1_1");
227-
File oldSnapshotDir = new File(regionDir, oldSnapshotDirName);
228-
if (oldSnapshotDir.exists()) {
229-
FileUtils.deleteFully(oldSnapshotDir);
230-
}
231-
if (!oldSnapshotDir.mkdirs()) {
232-
throw new ConsensusGroupModifyPeerException(
233-
String.format("%s: cannot mkdir for snapshot", dataRegionId));
234-
}
235-
consensusImpl.triggerSnapshot(dataRegionId, false);
236-
Assert.assertFalse(oldSnapshotDir.exists());
237-
238-
File dataDir = new File(IoTConsensus.buildPeerDir(storageDir, dataRegionId));
239-
240-
File[] snapshotFiles =
241-
dataDir.listFiles((dir, name) -> name.startsWith(IoTConsensusServerImpl.SNAPSHOT_DIR_NAME));
242-
Assert.assertNotNull(snapshotFiles);
243-
Assert.assertEquals(1, snapshotFiles.length);
244-
Assert.assertEquals(
245-
oldSnapshotIndex + 1,
246-
Long.parseLong(snapshotFiles[0].getName().replaceAll(".*[^\\d](?=(\\d+))", "")));
247-
consensusImpl.deleteLocalPeer(dataRegionId);
248-
}
249213
}

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/RegionMigrateService.java

Lines changed: 18 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -256,42 +256,29 @@ private TSStatus addPeer() {
256256
TEndPoint destEndpoint = getConsensusEndPoint(destDataNode, regionId);
257257
boolean addPeerSucceed = true;
258258
Throwable throwable = null;
259-
for (int i = 0; i < MAX_RETRY_NUM; i++) {
260-
try {
261-
if (!addPeerSucceed) {
262-
Thread.sleep(SLEEP_MILLIS);
263-
}
264-
addRegionPeer(regionId, new Peer(regionId, destDataNode.getDataNodeId(), destEndpoint));
265-
addPeerSucceed = true;
266-
} catch (PeerAlreadyInConsensusGroupException e) {
267-
addPeerSucceed = true;
268-
} catch (InterruptedException e) {
269-
throwable = e;
270-
Thread.currentThread().interrupt();
271-
} catch (ConsensusException e) {
272-
addPeerSucceed = false;
273-
throwable = e;
274-
taskLogger.error(
275-
"{}, executed addPeer {} for region {} error, retry times: {}",
276-
REGION_MIGRATE_PROCESS,
277-
destEndpoint,
278-
regionId,
279-
i,
280-
e);
281-
} catch (Exception e) {
282-
addPeerSucceed = false;
283-
throwable = e;
284-
taskLogger.warn("Unexpected exception", e);
285-
}
286-
if (addPeerSucceed || throwable instanceof InterruptedException) {
287-
break;
288-
}
259+
try {
260+
addRegionPeer(regionId, new Peer(regionId, destDataNode.getDataNodeId(), destEndpoint));
261+
} catch (PeerAlreadyInConsensusGroupException ignore) {
262+
263+
} catch (ConsensusException e) {
264+
addPeerSucceed = false;
265+
throwable = e;
266+
taskLogger.error(
267+
"{}, executed addPeer {} for region {} error",
268+
REGION_MIGRATE_PROCESS,
269+
destEndpoint,
270+
regionId,
271+
e);
272+
} catch (Exception e) {
273+
addPeerSucceed = false;
274+
throwable = e;
275+
taskLogger.warn("Unexpected exception", e);
289276
}
290277

291278
if (!addPeerSucceed) {
292279
String errorMsg =
293280
String.format(
294-
"%s, AddPeer for region error after max retry times, peerId: %s, regionId: %s",
281+
"%s, AddPeer for region error, peerId: %s, regionId: %s",
295282
REGION_MIGRATE_PROCESS, destEndpoint, regionId);
296283
taskLogger.error(errorMsg, throwable);
297284
status.setCode(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());

0 commit comments

Comments
 (0)