Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IoTConsensusV2: Fix some consensus group missing due to recover failed or blocked #14613

Merged
merged 3 commits into from
Jan 10, 2025
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -73,8 +73,11 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
Expand Down Expand Up @@ -123,7 +126,7 @@ public PipeConsensus(ConsensusConfig config, IStateMachine.Registry registry) {

@Override
public synchronized void start() throws IOException {
initAndRecover();
Future<Void> recoverFuture = initAndRecover();

rpcService.initSyncedServiceImpl(new PipeConsensusRPCServiceProcessor(this, config.getPipe()));
try {
Expand All @@ -132,19 +135,31 @@ public synchronized void start() throws IOException {
throw new IOException(e);
}

try {
recoverFuture.get();
} catch (CancellationException ce) {
LOGGER.info("IoTV2 Recover Task is cancelled", ce);
} catch (ExecutionException ee) {
LOGGER.error("Exception while waiting for recover future completion", ee);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
LOGGER.warn("IoTV2 Recover Task is interrupted", ie);
}
// only when we recover all consensus group can we launch async backend checker thread
consensusPipeGuardian.start(
CONSENSUS_PIPE_GUARDIAN_TASK_ID,
this::checkAllConsensusPipe,
config.getPipe().getConsensusPipeGuardJobIntervalInSeconds());
}

private void initAndRecover() throws IOException {
private Future<Void> initAndRecover() throws IOException {
if (!storageDir.exists()) {
// init
if (!storageDir.mkdirs()) {
LOGGER.warn("Unable to create consensus dir at {}", storageDir);
throw new IOException(String.format("Unable to create consensus dir at %s", storageDir));
}
return CompletableFuture.completedFuture(null);
} else {
// asynchronously recover, retry logic is implemented at PipeConsensusImpl
CompletableFuture<Void> future =
Expand All @@ -155,27 +170,39 @@ private void initAndRecover() throws IOException {
for (Path path : stream) {
ConsensusGroupId consensusGroupId =
parsePeerFileName(path.getFileName().toString());
PipeConsensusServerImpl consensus =
new PipeConsensusServerImpl(
new Peer(consensusGroupId, thisNodeId, thisNode),
registry.apply(consensusGroupId),
path.toString(),
new ArrayList<>(),
config,
consensusPipeManager,
syncClientManager);
stateMachineMap.put(consensusGroupId, consensus);
checkPeerListAndStartIfEligible(consensusGroupId, consensus);
try {
PipeConsensusServerImpl consensus =
new PipeConsensusServerImpl(
new Peer(consensusGroupId, thisNodeId, thisNode),
registry.apply(consensusGroupId),
path.toString(),
new ArrayList<>(),
config,
consensusPipeManager,
syncClientManager);
stateMachineMap.put(consensusGroupId, consensus);
checkPeerListAndStartIfEligible(consensusGroupId, consensus);
} catch (Exception e) {
LOGGER.error(
"Failed to recover consensus from {} for {}, ignore it and continue recover other group, async backend checker thread will automatically deregister related pipe side effects for this failed consensus group.",
storageDir,
consensusGroupId,
e);
}
}
} catch (Exception e) {
LOGGER.error("Failed to recover consensus from {}", storageDir, e);
} catch (IOException e) {
LOGGER.error(
"Failed to recover consensus from {} because read dir failed",
storageDir,
e);
}
})
.exceptionally(
e -> {
LOGGER.error("Failed to recover consensus from {}", storageDir, e);
return null;
});
return future;
}
}

Expand Down
Loading