public RaftConfiguration readRaftConfiguration() { File confFile = storageDir.getMetaConfFile(); try (FileInputStream fio = new FileInputStream(confFile)) { LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); return ServerProtoUtils.toRaftConfiguration(confProto); } catch (Exception e) { LOG.error("Failed reading configuration from file:" + confFile, e); return null; } }
void setRaftConf(LogEntryProto entry) { if (entry.hasConfigurationEntry()) { setRaftConf(entry.getIndex(), ServerProtoUtils.toRaftConfiguration(entry)); } }
void updateConfiguration(LogEntryProto[] entries) { if (entries != null && entries.length > 0) { configurationManager.removeConfigurations(entries[0].getIndex()); for (LogEntryProto entry : entries) { if (ProtoUtils.isConfigurationLogEntry(entry)) { final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration( entry.getIndex(), entry.getConfigurationEntry()); configurationManager.addConfiguration(entry.getIndex(), conf); server.getServerRpc().addPeers(conf.getPeers()); } } } }
CompletableFuture<Message> applyLogToStateMachine(LogEntryProto next) { final StateMachine stateMachine = getStateMachine(); if (next.getLogEntryBodyCase() == CONFIGURATIONENTRY) { // the reply should have already been set. only need to record // the new conf in the state machine. stateMachine.setRaftConfiguration(toRaftConfiguration(next.getIndex(), next.getConfigurationEntry())); } else if (next.getLogEntryBodyCase() == SMLOGENTRY) { // check whether there is a TransactionContext because we are the leader. TransactionContext trx = getTransactionContext(next.getIndex()); if (trx == null) { trx = new TransactionContextImpl(stateMachine, next); } // Let the StateMachine inject logic for committed transactions in sequential order. trx = stateMachine.applyTransactionSerial(trx); // TODO: This step can be parallelized CompletableFuture<Message> stateMachineFuture = stateMachine.applyTransaction(trx); return replyPendingRequest(next, stateMachineFuture); } return null; }
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); final File dir = RaftServerConfigKeys.storageDir(prop); storage = new RaftStorage(new File(dir, group.getGroupId().toString()), RaftServerConstants.StartupOption.REGULAR); snapshotManager = new SnapshotManager(storage, id); long lastApplied = initStatemachine(stateMachine, prop); leaderId = null; // 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, entry -> { if (entry.getLogEntryBodyCase() == CONFIGURATIONENTRY) { configurationManager.addConfiguration(entry.getIndex(), ServerProtoUtils.toRaftConfiguration(entry.getIndex(), entry.getConfigurationEntry())); } }); RaftLog.Metadata metadata = log.loadMetadata(); currentTerm = metadata.getTerm(); votedFor = metadata.getVotedFor(); stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log, lastApplied, prop); }