@Override public synchronized void close() { if (isClosed) { LOG.info("{} is already closed.", getId()); return; } isClosed = true; map.values().parallelStream().map(CompletableFuture::join) .forEach(impl -> impl.shutdown(false)); }
private void checkAndUpdateConfiguration(TermIndex[] entriesToCheck) { final RaftConfiguration conf = server.getRaftConf(); if (committedConf(entriesToCheck)) { if (conf.isTransitional()) { replicateNewConf(); } else { // the (new) log entry has been committed pendingRequests.replySetConfiguration(server::getCommitInfos); // if the leader is not included in the current configuration, step down if (!conf.containsInConf(server.getId())) { LOG.info("{} is not included in the new configuration {}. Step down.", server.getId(), conf); try { // leave some time for all RPC senders to send out new conf entry Thread.sleep(server.getMinTimeoutMs()); } catch (InterruptedException ignored) { } // the pending request handler will send NotLeaderException for // pending client requests when it stops server.shutdown(false); } } } }
private void checkAndUpdateConfiguration(TermIndex[] entriesToCheck) { final RaftConfiguration conf = server.getRaftConf(); if (committedConf(entriesToCheck)) { if (conf.isTransitional()) { replicateNewConf(); } else { // the (new) log entry has been committed LOG.debug("{} sends success to setConfiguration request", server.getId()); pendingRequests.replySetConfiguration(); // if the leader is not included in the current configuration, step down if (!conf.containsInConf(server.getId())) { LOG.info("{} is not included in the new configuration {}. Step down.", server.getId(), conf); try { // leave some time for all RPC senders to send out new conf entry Thread.sleep(server.getMinTimeoutMs()); } catch (InterruptedException ignored) { } // the pending request handler will send NotLeaderException for // pending client requests when it stops server.shutdown(); } } } }
private CompletableFuture<RaftClientReply> groupRemoveAsync( RaftClientRequest request, RaftGroupId groupId, boolean deleteDirectory) { if (!request.getRaftGroupId().equals(groupId)) { return JavaUtils.completeExceptionally(new GroupMismatchException( getId() + ": Request group id (" + request.getRaftGroupId() + ") does not match the given group id " + groupId)); } final CompletableFuture<RaftServerImpl> f = impls.remove(groupId); if (f == null) { return JavaUtils.completeExceptionally(new GroupMismatchException( getId() + ": Group " + groupId + " not found.")); } return f.thenApply(impl -> { final Collection<CommitInfoProto> commitInfos = impl.getCommitInfos(); impl.shutdown(deleteDirectory); return new RaftClientReply(request, commitInfos); }); }
Thread t = new Thread(() -> secondFollower.shutdown(true)); t.start();