private void checkInitialization() throws IOException { if (this.log == null) { ServerState state = proxy.getImpl(groupId).getState(); this.log = state.getLog(); } }
boolean isReady() { return server.getState().getLastAppliedIndex() >= placeHolderIndex; }
/** * Tests choosing of storage directory when only one volume is configured. * * @throws IOException in case of exception. */ @Test public void testChooseStorageDirWithOneVolume() throws IOException { File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString()); List<File> directories = Collections.singletonList(testDir); String subDirOne = UUID.randomUUID().toString(); String subDirTwo = UUID.randomUUID().toString(); File storageDirOne = ServerState.chooseStorageDir(directories, subDirOne); File storageDirTwo = ServerState.chooseStorageDir(directories, subDirTwo); File expectedOne = new File(testDir, subDirOne); File expectedTwo = new File(testDir, subDirTwo); Assert.assertEquals(expectedOne.getCanonicalPath(), storageDirOne.getCanonicalPath()); Assert.assertEquals(expectedTwo.getCanonicalPath(), storageDirTwo.getCanonicalPath()); }
private boolean containPrevious(TermIndex previous) { if (LOG.isTraceEnabled()) { LOG.trace("{}: prev:{}, latestSnapshot:{}, latestInstalledSnapshot:{}", getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot()); } return state.getLog().contains(previous) || (state.getLatestSnapshot() != null && state.getLatestSnapshot().getTermIndex().equals(previous)) || (state.getLatestInstalledSnapshot() != null) && state.getLatestInstalledSnapshot().equals(previous); }
boolean isConfCommitted() { return getLog().getLastCommittedIndex() >= getRaftConf().getLogEntryIndex(); }
@Override public void close() throws IOException { try { stateMachineUpdater.stopAndJoin(); } catch (InterruptedException e) { LOG.warn(getSelfId() + ": Interrupted when joining stateMachineUpdater", e); } LOG.info("{} closes. The last applied log index is {}", getSelfId(), getLastAppliedIndex()); log.close(); storage.close(); }
public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId, long oldLastApplied) throws Exception { long leaderApplied = cluster.getLeader().getState().getLastAppliedIndex(); // make sure retry cache has the entry for (RaftServerImpl server : cluster.iterateServerImpls()) { LOG.info("check server " + server.getId()); if (server.getState().getLastAppliedIndex() < leaderApplied) { Thread.sleep(1000); } Assert.assertEquals(2, RaftServerTestUtil.getRetryCacheSize(server)); Assert.assertNotNull(RaftServerTestUtil.getRetryEntry(server, clientId, callId)); // make sure there is only one log entry committed Assert.assertEquals(1, count(server.getState().getLog(), oldLastApplied + 1)); } }
private void applyOldNewConf() { final ServerState state = server.getState(); final RaftConfiguration current = server.getRaftConf(); final RaftConfiguration oldNewConf= stagingState.generateOldNewConf(current, state.getLog().getNextIndex()); // apply the (old, new) configuration to log, and use it as the current conf long index = state.getLog().append(state.getCurrentTerm(), oldNewConf); updateConfiguration(index, oldNewConf); this.stagingState = null; notifySenders(); }
final long lastIncludedIndex = lastTermIndex.getIndex(); synchronized (this) { final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { final InstallSnapshotReplyProto reply = ServerProtoUtils state.setLeader(leaderId, "installSnapshot"); state.getLog().getNextIndex() <= lastIncludedIndex, "%s log's next id is %s, last included index in snapshot is %s", getId(), state.getLog().getNextIndex(), lastIncludedIndex); state.installSnapshot(request); state.reloadStateMachine(lastIncludedIndex, leaderTerm);
LeaderState(RaftServerImpl server, RaftProperties properties) { this.server = server; stagingCatchupGap = RaftServerConfigKeys.stagingCatchupGap(properties); syncInterval = RaftServerConfigKeys.Rpc.sleepTime(properties); final ServerState state = server.getState(); this.raftLog = state.getLog(); this.currentTerm = state.getCurrentTerm(); eventQ = new ArrayBlockingQueue<>(4096); processor = new EventProcessor(); pendingRequests = new PendingRequests(server); final RaftConfiguration conf = server.getRaftConf(); Collection<RaftPeer> others = conf.getOtherPeers(state.getSelfId()); final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs()); placeHolderIndex = raftLog.getNextIndex(); senders = new SenderList(others.stream().map( p -> server.newLogAppender(this, p, t, placeHolderIndex, true)) .toArray(LogAppender[]::new)); voterLists = divideFollowers(conf); }
if (shouldWithholdVotes(candidateTerm)) { LOG.info("{}-{}: Withhold vote from candidate {} with term {}. State: leader={}, term={}, lastRpcElapsed={}", getId(), role, candidateId, candidateTerm, state.getLeaderId(), state.getCurrentTerm(), fs != null? fs.getLastRpcTime().elapsedTimeMs() + "ms": null); } else if (state.recognizeCandidate(candidateId, candidateTerm)) { final boolean termUpdated = changeToFollower(candidateTerm, true, "recognizeCandidate:" + candidateId); if (state.isLogUpToDate(candidateLastEntry) && fs != null) { fs.updateLastRpcTime(FollowerState.UpdateType.REQUEST_VOTE); state.grantVote(candidateId); voteGranted = true; state.persistMetadata(); // sync metafile groupId, voteGranted, state.getCurrentTerm(), shouldShutdown); if (LOG.isDebugEnabled()) { LOG.debug("{} replies to vote request: {}. Peer's state: {}",
Collection<CommitInfoProto> getCommitInfos() { final List<CommitInfoProto> infos = new ArrayList<>(); // add the commit info of this server infos.add(commitInfoCache.update(getPeer(), state.getLog().getLastCommittedIndex())); // add the commit infos of other servers if (isLeader()) { Optional.ofNullable(leaderState).ifPresent( leader -> leader.updateFollowerCommitInfos(commitInfoCache, infos)); } else { getRaftConf().getPeers().stream() .filter(p -> !p.getId().equals(state.getSelfId())) .map(RaftPeer::getId) .map(commitInfoCache::get) .filter(i -> i != null) .forEach(infos::add); } return infos; }
.getStorage().getStorageDir(); final long nextIndex = cluster.getLeader().getState().getLog().getNextIndex(); LOG.info("nextIndex = {}", nextIndex); final List<File> snapshotFiles = getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex);
void setLeader(RaftPeerId newLeaderId, String op) { if (!Objects.equals(leaderId, newLeaderId)) { LOG.info("{}: change Leader from {} to {} at term {} for {}", selfId, leaderId, newLeaderId, getCurrentTerm(), op); leaderId = newLeaderId; } }
ServerState(RaftPeerId id, RaftGroup group, RaftProperties prop, RaftServerImpl server, StateMachine stateMachine) throws IOException { this.selfId = id; this.server = server; RaftConfiguration initialConf = RaftConfiguration.newBuilder() .setConf(group.getPeers()).build(); configurationManager = new ConfigurationManager(initialConf); LOG.info("{}: {}", id, configurationManager); // use full uuid string to create a subdirectory final File dir = chooseStorageDir(RaftServerConfigKeys.storageDirs(prop), group.getGroupId().getUuid().toString()); storage = new RaftStorage(dir, RaftServerConstants.StartupOption.REGULAR); snapshotManager = new SnapshotManager(storage, id); long lastApplied = initStatemachine(stateMachine, group.getGroupId()); // On start the leader is null, start the clock now leaderId = null; this.lastNoLeaderTime = Timestamp.currentTime(); this.leaderElectionTimeoutMs = RaftServerConfigKeys.leaderElectionTimeout(prop).toIntExact(TimeUnit.MILLISECONDS); // we cannot apply log entries to the state machine in this step, since we // do not know whether the local log entries have been committed. log = initLog(id, prop, lastApplied, this::setRaftConf); RaftLog.Metadata metadata = log.loadMetadata(); currentTerm = metadata.getTerm(); votedFor = metadata.getVotedFor(); stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log, lastApplied, prop); }
static List<Path> getOpenLogFiles(RaftServerImpl server) throws Exception { return server.getState().getStorage().getStorageDir().getLogSegmentFiles().stream() .filter(LogPathAndIndex::isOpen) .map(LogPathAndIndex::getPath) .collect(Collectors.toList()); }
private boolean shouldWithholdVotes(long candidateTerm) { if (state.getCurrentTerm() < candidateTerm) { return false; } else if (isLeader()) { return true; } else { // following a leader and not yet timeout return isFollower() && state.hasLeader() && role.getFollowerState().map(FollowerState::shouldWithholdVotes).orElse(false); } }
NotLeaderException generateNotLeaderException() { if (lifeCycle.getCurrentState() != RUNNING) { return new NotLeaderException(getId(), null, null); } RaftPeerId leaderId = state.getLeaderId(); if (leaderId == null || leaderId.equals(state.getSelfId())) { // No idea about who is the current leader. Or the peer is the current // leader, but it is about to step down RaftPeer suggestedLeader = getRaftConf().getRandomPeer(state.getSelfId()); leaderId = suggestedLeader == null ? null : suggestedLeader.getId(); } RaftConfiguration conf = getRaftConf(); Collection<RaftPeer> peers = conf.getPeers(); return new NotLeaderException(getId(), conf.getPeer(leaderId), peers.toArray(new RaftPeer[peers.size()])); }
state.close(); } catch (Exception ignored) { LOG.warn("Failed to close state for " + getId(), ignored); final RaftStorageDirectory dir = state.getStorage().getStorageDir(); try { FileUtils.deleteFully(dir.getRoot());
static RaftPeerId getLeader(ServerState state) { final RaftPeerId leader = state.getLeaderId(); if (leader == null) { throw new IllegalStateException("No leader yet"); } return leader; } }