void start() { lifeCycle.transition(STARTING); state.start(); RaftConfiguration conf = getRaftConf(); if (conf != null && conf.contains(getId())) { LOG.debug("{} starts as a follower, conf={}", getId(), conf); startAsFollower(); } else { LOG.debug("{} starts with initializing state, conf={}", getId(), conf); startInitializing(); } registerMBean(getId(), getGroupId(), jmxAdapter, jmxAdapter); }
private void checkAndEvictCache() { if (server.isPresent() && cache.shouldEvict()) { // TODO if the cache is hitting the maximum size and we cannot evict any // segment's cache, should block the new entry appending or new segment // allocation. final RaftServerImpl s = server.get(); cache.evictCache(s.getFollowerNextIndices(), fileLogWorker.getFlushedIndex(), s.getState().getLastAppliedIndex()); } }
public GrpcLogAppender(RaftServerImpl server, LeaderState leaderState, FollowerInfo f) { super(server, leaderState, f); this.rpcService = (GrpcService) server.getServerRpc(); maxPendingRequestsNum = GrpcConfigKeys.Server.leaderOutstandingAppendsMax( server.getProxy().getProperties()); requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(server.getProxy().getProperties()); pendingRequests = new ConcurrentHashMap<>(); }
@Override public RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException { return waitForReply(getId(), request, setConfigurationAsync(request)); }
LeaderElection(RaftServerImpl server) { this.server = server; conf = server.getRaftConf(); others = conf.getOtherPeers(server.getId()); this.running = true; }
public CompletableFuture<RaftClientReply> setConfigurationAsync( SetConfigurationRequest request) throws IOException { LOG.debug("{}: receive setConfiguration({})", getId(), request); assertLifeCycleState(RUNNING); assertGroup(request.getRequestorId(), request.getRaftGroupId()); CompletableFuture<RaftClientReply> reply = checkLeaderState(request, null); if (reply != null) { return reply; final PendingRequest pending; synchronized (this) { reply = checkLeaderState(request, null); if (reply != null) { return reply; final RaftConfiguration current = getRaftConf(); pending.setReply(new RaftClientReply(request, getCommitInfos())); return pending.getFuture(); getServerRpc().addPeers(Arrays.asList(peersInNewConf));
private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot) throws InterruptedIOException { String requestId = UUID.randomUUID().toString(); InstallSnapshotReplyProto reply = null; try { for (InstallSnapshotRequestProto request : new SnapshotRequestIter(snapshot, requestId)) { follower.updateLastRpcSendTime(); reply = server.getServerRpc().installSnapshot(request); follower.updateLastRpcResponseTime(); if (!reply.getServerReply().getSuccess()) { return reply; } } } catch (InterruptedIOException iioe) { throw iioe; } catch (Exception ioe) { LOG.warn("{}: Failed to installSnapshot {}: {}", this, snapshot, ioe); handleException(ioe); return null; } if (reply != null) { follower.setSnapshotIndex(snapshot.getTermIndex().getIndex()); LOG.info("{}: install snapshot-{} successfully on follower {}", server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer()); } return reply; }
final ServerState state = server.getState(); while (running && server.isCandidate()) { server.getState().persistMetadata(); !server.isCandidate()) { return; // term already passed or no longer a candidate. server.changeToLeader(); return; case SHUTDOWN: LOG.info("{} received shutdown response when requesting votes.", server.getId()); server.getProxy().close(); return; case REJECTED: case DISCOVERED_A_NEW_TERM: final long term = r.term > server.getState().getCurrentTerm() ? r.term : server.getState().getCurrentTerm(); server.changeToFollowerAndPersistMetadata(term, Result.DISCOVERED_A_NEW_TERM); return; case TIMEOUT:
final ServerState state = server.getState(); while (running && server.isCandidate()) { server.getState().persistMetadata(); !server.isCandidate()) { return; // term already passed or no longer a candidate. server.changeToLeader(); return; case SHUTDOWN: LOG.info("{} received shutdown response when requesting votes.", server.getId()); server.getProxy().close(); return; case REJECTED: case DISCOVERED_A_NEW_TERM: final long term = r.term > server.getState().getCurrentTerm() ? r.term : server.getState().getCurrentTerm(); server.changeToFollower(term, true); return; case TIMEOUT:
@Test public void testServerRestartOnException() throws Exception { RaftProperties properties = new RaftProperties(); final MiniRaftClusterWithGRpc cluster = MiniRaftClusterWithGRpc.FACTORY.newCluster(1, properties); cluster.start(); RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); GrpcConfigKeys.Server.setPort(properties, cluster.getLeader().getServerRpc().getInetSocketAddress().getPort()); // Create a raft server proxy with server rpc bound to a different address // compared to leader. This helps in locking the raft storage directory to // be used by next raft server proxy instance. final StateMachine stateMachine = cluster.getLeader().getStateMachine(); ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), stateMachine, properties, null); // Close the server rpc for leader so that new raft server can be bound to it. cluster.getLeader().getServerRpc().close(); // Create a raft server proxy with server rpc bound to same address as // the leader. This step would fail as the raft storage has been locked by // the raft server proxy created earlier. Raft server proxy should close // the rpc server on failure. testFailureCase("start a new server with the same address", () -> ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), stateMachine, properties, null), IOException.class, IOException.class, OverlappingFileLockException.class); // Try to start a raft server rpc at the leader address. cluster.getServer(leaderId).getFactory().newRaftServerRpc(cluster.getServer(leaderId)); } }
void submit(StateUpdateEvent event) { try { queue.put(event); } catch (InterruptedException e) { LOG.info("{}: Interrupted when submitting {} ", server.getId(), event); } }
public void failClientRequest(LogEntryProto logEntry) { if (logEntry.getLogEntryBodyCase() == LogEntryProto.LogEntryBodyCase.SMLOGENTRY) { final ClientId clientId = ClientId.valueOf(logEntry.getClientId()); final RetryCache.CacheEntry cacheEntry = getRetryCache().get(clientId, logEntry.getCallId()); if (cacheEntry != null) { final RaftClientReply reply = new RaftClientReply(clientId, getId(), getGroupId(), logEntry.getCallId(), false, null, generateNotLeaderException(), getCommitInfos()); cacheEntry.failWithReply(reply); } } }
/** * @return null if the server is in leader state. */ private CompletableFuture<RaftClientReply> checkLeaderState( RaftClientRequest request, RetryCache.CacheEntry entry) { try { assertGroup(request.getRequestorId(), request.getRaftGroupId()); } catch (GroupMismatchException e) { return RetryCache.failWithException(e, entry); } if (!isLeader()) { NotLeaderException exception = generateNotLeaderException(); final RaftClientReply reply = new RaftClientReply(request, exception, getCommitInfos()); return RetryCache.failWithReply(reply, entry); } else if (leaderState == null || !leaderState.isReady()) { RetryCache.CacheEntry cacheEntry = retryCache.get(request.getClientId(), request.getCallId()); if (cacheEntry != null && cacheEntry.isCompletedNormally()) { return cacheEntry.getReplyFuture(); } return RetryCache.failWithException(new LeaderNotReadyException(getId()), entry); } return null; }
private CompletableFuture<RaftClientReply> staleReadAsync(RaftClientRequest request) { final long minIndex = request.getType().getStaleRead().getMinIndex(); final long commitIndex = state.getLog().getLastCommittedIndex(); LOG.debug("{}: minIndex={}, commitIndex={}", getId(), minIndex, commitIndex); if (commitIndex < minIndex) { final StaleReadException e = new StaleReadException( "Unable to serve stale-read due to server commit index = " + commitIndex + " < min = " + minIndex); return CompletableFuture.completedFuture( new RaftClientReply(request, new StateMachineException(getId(), e), getCommitInfos())); } return processQueryFuture(getStateMachine().queryStale(request.getMessage(), minIndex), request); }
/** * check if the remote peer is not included in the current conf * and should shutdown. should shutdown if all the following stands: * 1. this is a leader * 2. current conf is stable and has been committed * 3. candidate id is not included in conf * 4. candidate's last entry's index < conf's index */ private boolean shouldSendShutdown(RaftPeerId candidateId, TermIndex candidateLastEntry) { return isLeader() && getRaftConf().isStable() && getState().isConfCommitted() && !getRaftConf().containsInConf(candidateId) && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex() && !leaderState.isBootStrappingPeer(candidateId); }
private void prepare() { synchronized (server) { if (running) { final RaftConfiguration conf = server.getRaftConf(); if (conf.isTransitional() && server.getState().isConfCommitted()) { // the configuration is in transitional state, and has been committed // so it is time to generate and replicate (new) conf. replicateNewConf(); } } } }
void stop() { this.running = false; // do not interrupt event processor since it may be in the middle of logSync senders.forEach(LogAppender::stopAppender); final NotLeaderException nle = server.generateNotLeaderException(); final Collection<CommitInfoProto> commitInfos = server.getCommitInfos(); try { final Collection<TransactionContext> transactions = pendingRequests.sendNotLeaderResponses(nle, commitInfos); server.getStateMachine().notifyNotLeader(transactions); watchRequests.failWatches(nle); } catch (IOException e) { LOG.warn(server.getId() + ": Caught exception in sendNotLeaderResponses", e); } }
public LogAppender(RaftServerImpl server, LeaderState leaderState, FollowerInfo f) { this.follower = f; this.server = server; this.leaderState = leaderState; this.raftLog = server.getState().getLog(); final RaftProperties properties = server.getProxy().getProperties(); this.snapshotChunkMaxSize = RaftServerConfigKeys.Log.Appender.snapshotChunkSizeMax(properties).getSizeInt(); this.halfMinTimeoutMs = server.getMinTimeoutMs() / 2; final SizeInBytes bufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); final int bufferElementLimit = RaftServerConfigKeys.Log.Appender.bufferElementLimit(properties); this.buffer = new DataQueue<>(this, bufferByteLimit, bufferElementLimit, EntryWithData::getSerializedSize); this.lifeCycle = new LifeCycle(this); }
static void assertLogEntries(RaftServerImpl server, long expectedTerm, SimpleMessage... expectedMessages) { LOG.info("checking raft log for " + server.getId()); final RaftLog log = server.getState().getLog(); try { RaftTestUtil.assertLogEntries(log, expectedTerm, expectedMessages); } catch (AssertionError e) { LOG.error(server.getId() + ": Unexpected raft log", e); throw e; } }
private void checkInitialization() throws IOException { if (this.log == null) { ServerState state = proxy.getImpl(groupId).getState(); this.log = state.getLog(); } }