未验证 提交 b17e9937 编写于 作者: L Li Minghao 提交者: GitHub

Merge pull request #10162 from taosdata/feature/3.0_mhli

Feature/3.0 mhli
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_H
#define _TD_LIBS_SYNC_H
#ifdef __cplusplus
extern "C" {
#endif
#include <stdint.h>
#include "taosdef.h"
typedef int32_t SyncNodeId;
typedef int32_t SyncGroupId;
typedef int64_t SyncIndex;
typedef uint64_t SyncTerm;
typedef enum {
TAOS_SYNC_STATE_FOLLOWER = 0,
TAOS_SYNC_STATE_CANDIDATE = 1,
TAOS_SYNC_STATE_LEADER = 2,
} ESyncState;
typedef struct {
void* data;
size_t len;
} SSyncBuffer;
typedef struct {
SyncNodeId nodeId;
uint16_t nodePort; // node sync Port
char nodeFqdn[TSDB_FQDN_LEN]; // node FQDN
} SNodeInfo;
typedef struct {
int32_t selfIndex;
int32_t replica;
SNodeInfo nodeInfo[TSDB_MAX_REPLICA];
} SSyncCluster;
typedef struct {
int32_t selfIndex;
int32_t replica;
SNodeInfo node[TSDB_MAX_REPLICA];
ESyncState role[TSDB_MAX_REPLICA];
} SNodesRole;
typedef struct SSyncFSM {
void* pData;
// apply committed log, bufs will be free by sync module
int32_t (*applyLog)(struct SSyncFSM* fsm, SyncIndex index, const SSyncBuffer* buf, void* pData);
// cluster commit callback
int32_t (*onClusterChanged)(struct SSyncFSM* fsm, const SSyncCluster* cluster, void* pData);
// fsm return snapshot in ppBuf, bufs will be free by sync module
// TODO: getSnapshot SHOULD be async?
int32_t (*getSnapshot)(struct SSyncFSM* fsm, SSyncBuffer** ppBuf, int32_t* objId, bool* isLast);
// fsm apply snapshot with pBuf data
int32_t (*applySnapshot)(struct SSyncFSM* fsm, SSyncBuffer* pBuf, int32_t objId, bool isLast);
// call when restore snapshot and log done
int32_t (*onRestoreDone)(struct SSyncFSM* fsm);
void (*onRollback)(struct SSyncFSM* fsm, SyncIndex index, const SSyncBuffer* buf);
void (*onRoleChanged)(struct SSyncFSM* fsm, const SNodesRole* pRole);
} SSyncFSM;
typedef struct SSyncLogStore {
void* pData;
// write log with given index
int32_t (*logWrite)(struct SSyncLogStore* logStore, SyncIndex index, SSyncBuffer* pBuf);
/**
* read log from given index(included) with limit, return the actual num in nBuf,
* pBuf will be free in sync module
**/
int32_t (*logRead)(struct SSyncLogStore* logStore, SyncIndex index, int limit,
SSyncBuffer* pBuf, int* nBuf);
// mark log with given index has been commtted
int32_t (*logCommit)(struct SSyncLogStore* logStore, SyncIndex index);
// prune log before given index(not included)
int32_t (*logPrune)(struct SSyncLogStore* logStore, SyncIndex index);
// rollback log after given index(included)
int32_t (*logRollback)(struct SSyncLogStore* logStore, SyncIndex index);
// return last index of log
SyncIndex (*logLastIndex)(struct SSyncLogStore* logStore);
} SSyncLogStore;
typedef struct SStateManager {
void* pData;
// save serialized server state data, buffer will be free by Sync
int32_t (*saveServerState)(struct SStateManager* stateMng, const char* buffer, int n);
// read serialized server state data, buffer will be free by Sync
int32_t (*readServerState)(struct SStateManager* stateMng, char** ppBuffer, int* n);
// save serialized cluster state data, buffer will be free by Sync
void (*saveClusterState)(struct SStateManager* stateMng, const char* buffer, int n);
// read serialized cluster state data, buffer will be free by Sync
int32_t (*readClusterState)(struct SStateManager* stateMng, char** ppBuffer, int* n);
} SStateManager;
typedef struct {
SyncGroupId vgId;
SyncIndex appliedIndex;
SSyncCluster syncCfg;
SSyncFSM fsm;
SSyncLogStore logStore;
SStateManager stateManager;
} SSyncInfo;
struct SSyncNode;
typedef struct SSyncNode SSyncNode;
int32_t syncInit();
void syncCleanUp();
SSyncNode* syncStart(const SSyncInfo*);
void syncReconfig(const SSyncNode*, const SSyncCluster*);
void syncStop(const SSyncNode*);
int32_t syncPropose(SSyncNode* syncNode, const SSyncBuffer* pBuf, void* pData, bool isWeak);
int32_t syncAddNode(SSyncNode syncNode, const SNodeInfo *pNode);
int32_t syncRemoveNode(SSyncNode syncNode, const SNodeInfo *pNode);
extern int32_t sDebugFlag;
#ifdef __cplusplus
}
#endif
#endif /*_TD_LIBS_SYNC_H*/
aux_source_directory(src SYNC_SRC)
add_library(sync ${SYNC_SRC})
target_link_libraries(
sync
PUBLIC common
PUBLIC transport
PUBLIC util
PUBLIC wal
)
target_include_directories(
sync
PUBLIC "${CMAKE_SOURCE_DIR}/include/libs/sync"
PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc"
)
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_H
#define _TD_LIBS_SYNC_RAFT_H
#include "sync.h"
#include "sync_type.h"
#include "thash.h"
#include "raft_message.h"
#include "sync_raft_impl.h"
#include "sync_raft_quorum.h"
typedef struct RaftLeaderState {
} RaftLeaderState;
typedef struct RaftCandidateState {
/* true if in pre-vote phase */
bool inPreVote;
} RaftCandidateState;
typedef struct SSyncRaftIOMethods {
// send SSyncMessage to node
int (*send)(const SSyncMessage* pMsg, const SNodeInfo* pNode);
} SSyncRaftIOMethods;
typedef int (*SyncRaftStepFp)(SSyncRaft* pRaft, const SSyncMessage* pMsg);
typedef void (*SyncRaftTickFp)(SSyncRaft* pRaft);
struct SSyncRaft {
// owner sync node
SSyncNode* pNode;
// hash map nodeId -> SNodeInfo*
SHashObj* nodeInfoMap;
SyncNodeId selfId;
SyncGroupId selfGroupId;
SSyncRaftIOMethods io;
SSyncFSM fsm;
SSyncLogStore logStore;
SStateManager stateManager;
union {
RaftLeaderState leaderState;
RaftCandidateState candidateState;
};
SyncTerm term;
SyncNodeId voteFor;
SSyncRaftLog *log;
uint64_t maxMsgSize;
uint64_t maxUncommittedSize;
SSyncRaftProgressTracker *tracker;
ESyncState state;
// isLearner is true if the local raft node is a learner.
bool isLearner;
/**
* the leader id
**/
SyncNodeId leaderId;
/**
* leadTransferee is id of the leader transfer target when its value is not zero.
* Follow the procedure defined in raft thesis 3.10.
**/
SyncNodeId leadTransferee;
/**
* Only one conf change may be pending (in the log, but not yet
* applied) at a time. This is enforced via pendingConfIndex, which
* is set to a value >= the log index of the latest pending
* configuration change (if any). Config changes are only allowed to
* be proposed if the leader's applied index is greater than this
* value.
**/
SyncIndex pendingConfigIndex;
/**
* an estimate of the size of the uncommitted tail of the Raft log. Used to
* prevent unbounded log growth. Only maintained by the leader. Reset on
* term changes.
**/
uint32_t uncommittedSize;
/**
* number of ticks since it reached last electionTimeout when it is leader
* or candidate.
* number of ticks since it reached last electionTimeout or received a
* valid message from current leader when it is a follower.
**/
uint16_t electionElapsed;
/**
* number of ticks since it reached last heartbeatTimeout.
* only leader keeps heartbeatElapsed.
**/
uint16_t heartbeatElapsed;
bool preVote;
bool checkQuorum;
int heartbeatTimeout;
int electionTimeout;
/**
* randomizedElectionTimeout is a random number between
* [electiontimeout, 2 * electiontimeout - 1]. It gets reset
* when raft changes its state to follower or candidate.
**/
int randomizedElectionTimeout;
bool disableProposalForwarding;
// current tick count since start up
uint32_t currentTick;
SyncRaftStepFp stepFp;
SyncRaftTickFp tickFp;
};
int32_t syncRaftStart(SSyncRaft* pRaft, const SSyncInfo* pInfo);
int32_t syncRaftStep(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int32_t syncRaftTick(SSyncRaft* pRaft);
#endif /* _TD_LIBS_SYNC_RAFT_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_LOG_H
#define _TD_LIBS_SYNC_RAFT_LOG_H
#include "sync.h"
#include "sync_type.h"
typedef enum ESyncRaftEntryType {
SYNC_ENTRY_TYPE_LOG = 1,
} ESyncRaftEntryType;
struct SSyncRaftEntry {
SyncTerm term;
SyncIndex index;
ESyncRaftEntryType type;
SSyncBuffer buffer;
};
struct SSyncRaftLog {
SyncIndex uncommittedConfigIndex;
SyncIndex commitIndex;
SyncIndex appliedIndex;
};
SSyncRaftLog* syncRaftLogOpen();
SyncIndex syncRaftLogLastIndex(SSyncRaftLog* pLog);
SyncIndex syncRaftLogSnapshotIndex(SSyncRaftLog* pLog);
SyncTerm syncRaftLogLastTerm(SSyncRaftLog* pLog);
void syncRaftLogAppliedTo(SSyncRaftLog* pLog, SyncIndex appliedIndex);
bool syncRaftLogIsUptodate(SSyncRaftLog* pLog, SyncIndex index, SyncTerm term);
int syncRaftLogNumOfPendingConf(SSyncRaftLog* pLog);
bool syncRaftHasUnappliedLog(SSyncRaftLog* pLog);
SyncTerm syncRaftLogTermOf(SSyncRaftLog* pLog, SyncIndex index);
int syncRaftLogAppend(SSyncRaftLog* pLog, SSyncRaftEntry *pEntries, int n);
int syncRaftLogAcquire(SSyncRaftLog* pLog, SyncIndex index, int maxMsgSize,
SSyncRaftEntry **ppEntries, int *n);
void syncRaftLogRelease(SSyncRaftLog* pLog, SyncIndex index,
SSyncRaftEntry *pEntries, int n);
bool syncRaftLogMatchTerm();
static FORCE_INLINE bool syncRaftLogIsCommitted(SSyncRaftLog* pLog, SyncIndex index) {
return pLog->commitIndex > index;
}
#endif /* _TD_LIBS_SYNC_RAFT_LOG_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_MESSAGE_H
#define _TD_LIBS_SYNC_RAFT_MESSAGE_H
#include "sync.h"
#include "sync_type.h"
/**
* below define message type which handled by Raft.
*
* internal message, which communicate between threads, start with RAFT_MSG_INTERNAL_*.
* internal message use pointer only and stack memory, need not to be decode/encode and free.
*
* outter message start with RAFT_MSG_*, which communicate between cluster peers,
* need to implement its decode/encode functions.
**/
typedef enum ESyncRaftMessageType {
// client propose a cmd
RAFT_MSG_INTERNAL_PROP = 1,
// node election timeout
RAFT_MSG_INTERNAL_ELECTION = 2,
RAFT_MSG_VOTE = 3,
RAFT_MSG_VOTE_RESP = 4,
RAFT_MSG_APPEND = 5,
RAFT_MSG_APPEND_RESP = 6,
} ESyncRaftMessageType;
typedef struct RaftMsgInternal_Prop {
const SSyncBuffer *pBuf;
bool isWeak;
void* pData;
} RaftMsgInternal_Prop;
typedef struct RaftMsgInternal_Election {
} RaftMsgInternal_Election;
typedef struct RaftMsg_Vote {
ESyncRaftElectionType cType;
SyncIndex lastIndex;
SyncTerm lastTerm;
} RaftMsg_Vote;
typedef struct RaftMsg_VoteResp {
bool rejected;
ESyncRaftElectionType cType;
} RaftMsg_VoteResp;
typedef struct RaftMsg_Append_Entries {
// index of log entry preceeding new ones
SyncIndex index;
// term of entry at prevIndex
SyncTerm term;
// leader's commit index.
SyncIndex commitIndex;
// size of the log entries array
int nEntries;
// log entries array
SSyncRaftEntry* entries;
} RaftMsg_Append_Entries;
typedef struct RaftMsg_Append_Resp {
SyncIndex index;
} RaftMsg_Append_Resp;
typedef struct SSyncMessage {
ESyncRaftMessageType msgType;
SyncTerm term;
SyncGroupId groupId;
SyncNodeId from;
union {
RaftMsgInternal_Prop propose;
RaftMsgInternal_Election election;
RaftMsg_Vote vote;
RaftMsg_VoteResp voteResp;
RaftMsg_Append_Entries appendEntries;
RaftMsg_Append_Resp appendResp;
};
} SSyncMessage;
static FORCE_INLINE SSyncMessage* syncInitPropMsg(SSyncMessage* pMsg, const SSyncBuffer* pBuf, void* pData, bool isWeak) {
*pMsg = (SSyncMessage) {
.msgType = RAFT_MSG_INTERNAL_PROP,
.term = 0,
.propose = (RaftMsgInternal_Prop) {
.isWeak = isWeak,
.pBuf = pBuf,
.pData = pData,
},
};
return pMsg;
}
static FORCE_INLINE SSyncMessage* syncInitElectionMsg(SSyncMessage* pMsg, SyncNodeId from) {
*pMsg = (SSyncMessage) {
.msgType = RAFT_MSG_INTERNAL_ELECTION,
.term = 0,
.from = from,
.election = (RaftMsgInternal_Election) {
},
};
return pMsg;
}
static FORCE_INLINE SSyncMessage* syncNewVoteMsg(SyncGroupId groupId, SyncNodeId from,
SyncTerm term, ESyncRaftElectionType cType,
SyncIndex lastIndex, SyncTerm lastTerm) {
SSyncMessage* pMsg = (SSyncMessage*)malloc(sizeof(SSyncMessage));
if (pMsg == NULL) {
return NULL;
}
*pMsg = (SSyncMessage) {
.groupId = groupId,
.from = from,
.term = term,
.msgType = RAFT_MSG_VOTE,
.vote = (RaftMsg_Vote) {
.cType = cType,
.lastIndex = lastIndex,
.lastTerm = lastTerm,
},
};
return pMsg;
}
static FORCE_INLINE SSyncMessage* syncNewVoteRespMsg(SyncGroupId groupId, SyncNodeId from,
ESyncRaftElectionType cType, bool rejected) {
SSyncMessage* pMsg = (SSyncMessage*)malloc(sizeof(SSyncMessage));
if (pMsg == NULL) {
return NULL;
}
*pMsg = (SSyncMessage) {
.groupId = groupId,
.from = from,
.msgType = RAFT_MSG_VOTE_RESP,
.voteResp = (RaftMsg_VoteResp) {
.cType = cType,
.rejected = rejected,
},
};
return pMsg;
}
static FORCE_INLINE SSyncMessage* syncNewAppendMsg(SyncGroupId groupId, SyncNodeId from,
SyncTerm term, SyncIndex logIndex, SyncTerm logTerm,
SyncIndex commitIndex, int nEntries, SSyncRaftEntry* entries) {
SSyncMessage* pMsg = (SSyncMessage*)malloc(sizeof(SSyncMessage));
if (pMsg == NULL) {
return NULL;
}
*pMsg = (SSyncMessage) {
.groupId = groupId,
.from = from,
.term = term,
.msgType = RAFT_MSG_APPEND,
.appendEntries = (RaftMsg_Append_Entries) {
.index = logIndex,
.term = logTerm,
.commitIndex = commitIndex,
.nEntries = nEntries,
.entries = entries,
},
};
return pMsg;
}
static FORCE_INLINE SSyncMessage* syncNewEmptyAppendRespMsg(SyncGroupId groupId, SyncNodeId from, SyncTerm term) {
SSyncMessage* pMsg = (SSyncMessage*)malloc(sizeof(SSyncMessage));
if (pMsg == NULL) {
return NULL;
}
*pMsg = (SSyncMessage) {
.groupId = groupId,
.from = from,
.term = term,
.msgType = RAFT_MSG_APPEND_RESP,
.appendResp = (RaftMsg_Append_Resp) {
},
};
return pMsg;
}
static FORCE_INLINE bool syncIsInternalMsg(ESyncRaftMessageType msgType) {
return msgType == RAFT_MSG_INTERNAL_PROP ||
msgType == RAFT_MSG_INTERNAL_ELECTION;
}
static FORCE_INLINE bool syncIsPreVoteRespMsg(const SSyncMessage* pMsg) {
return pMsg->msgType == RAFT_MSG_VOTE_RESP && pMsg->voteResp.cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION;
}
static FORCE_INLINE bool syncIsPreVoteMsg(const SSyncMessage* pMsg) {
return pMsg->msgType == RAFT_MSG_VOTE && pMsg->voteResp.cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION;
}
void syncFreeMessage(const SSyncMessage* pMsg);
// message handlers
int syncRaftHandleElectionMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int syncRaftHandleVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int syncRaftHandleVoteRespMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int syncRaftHandleAppendEntriesMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
#endif /* _TD_LIBS_SYNC_RAFT_MESSAGE_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_REPLICATION_H
#define TD_SYNC_RAFT_REPLICATION_H
#include "sync.h"
#include "syncInt.h"
#include "sync_type.h"
// syncRaftMaybeSendAppend sends an append RPC with new entries to the given peer,
// if necessary. Returns true if a message was sent. The sendIfEmpty
// argument controls whether messages with no entries will be sent
// ("empty" messages are useful to convey updated Commit indexes, but
// are undesirable when we're sending multiple messages in a batch).
bool syncRaftMaybeSendAppend(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty);
#endif /* TD_SYNC_RAFT_REPLICATION_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_UNSTABLE_LOG_H
#define TD_SYNC_RAFT_UNSTABLE_LOG_H
#include "sync_type.h"
/* in-memory unstable raft log storage */
struct SSyncRaftUnstableLog {
#if 0
/* Circular buffer of log entries */
RaftEntry *entries;
/* size of Circular buffer */
int size;
/* Indexes of used slots [front, back) */
int front, back;
/* Index of first entry is offset + 1 */
SyncIndex offset;
/* meta data of snapshot */
SSyncRaftUnstableLog snapshot;
#endif
};
/**
* return index of last in memory log, return 0 if log is empty
**/
//SyncIndex syncRaftLogLastIndex(SSyncRaftUnstableLog* pLog);
#if 0
void raftLogInit(RaftLog* pLog);
void raftLogClose(RaftLog* pLog);
/**
* When startup populating log entrues loaded from disk,
* init raft memory log with snapshot index,term and log start idnex.
**/
/*
void raftLogStart(RaftLog* pLog,
RaftSnapshotMeta snapshot,
SyncIndex startIndex);
*/
/**
* Get the number of entries the log.
**/
int raftLogNumEntries(const RaftLog* pLog);
/**
* return last term of in memory log, return 0 if log is empty
**/
SyncTerm raftLogLastTerm(RaftLog* pLog);
/**
* return term of log with the given index, return 0 if the term of index cannot be found
* , errCode will save the error code.
**/
SyncTerm raftLogTermOf(RaftLog* pLog, SyncIndex index, RaftCode* errCode);
/**
* Get the last index of the most recent snapshot. Return 0 if there are no *
* snapshots.
**/
SyncIndex raftLogSnapshotIndex(RaftLog* pLog);
/* Append a new entry to the log. */
int raftLogAppend(RaftLog* pLog,
SyncTerm term,
const SSyncBuffer *buf);
/**
* acquire log from given index onwards.
**/
/*
int raftLogAcquire(RaftLog* pLog,
SyncIndex index,
RaftEntry **ppEntries,
int *n);
void raftLogRelease(RaftLog* pLog,
SyncIndex index,
RaftEntry *pEntries,
int n);
*/
/* Delete all entries from the given index (included) onwards. */
void raftLogTruncate(RaftLog* pLog, SyncIndex index);
/**
* when taking a new snapshot, the function will update the last snapshot information and delete
* all entries up last_index - trailing (included). If the log contains no entry
* a last_index - trailing, then no entry will be deleted.
**/
void raftLogSnapshot(RaftLog* pLog, SyncIndex index, SyncIndex trailing);
#endif
#endif /* TD_SYNC_RAFT_UNSTABLE_LOG_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_INT_H
#define _TD_LIBS_SYNC_INT_H
#include "thash.h"
#include "os.h"
#include "sync.h"
#include "sync_type.h"
#include "raft.h"
#include "tlog.h"
#define TAOS_SYNC_MAX_WORKER 3
typedef struct SSyncWorker {
pthread_t thread;
} SSyncWorker;
struct SSyncNode {
pthread_mutex_t mutex;
int32_t refCount;
SyncGroupId vgId;
SSyncRaft raft;
void* syncTimer;
};
typedef struct SSyncManager {
pthread_mutex_t mutex;
// sync server rpc
void* serverRpc;
// rpc server hash table base on FQDN:port key
SHashObj* rpcServerTable;
// sync client rpc
void* clientRpc;
// worker threads
SSyncWorker worker[TAOS_SYNC_MAX_WORKER];
// vgroup hash table
SHashObj* vgroupTable;
// timer manager
void* syncTimerManager;
} SSyncManager;
extern SSyncManager* gSyncManager;
#define syncFatal(...) do { if (sDebugFlag & DEBUG_FATAL) { taosPrintLog("SYNC FATAL ", 255, __VA_ARGS__); }} while(0)
#define syncError(...) do { if (sDebugFlag & DEBUG_ERROR) { taosPrintLog("SYNC ERROR ", 255, __VA_ARGS__); }} while(0)
#define syncWarn(...) do { if (sDebugFlag & DEBUG_WARN) { taosPrintLog("SYNC WARN ", 255, __VA_ARGS__); }} while(0)
#define syncInfo(...) do { if (sDebugFlag & DEBUG_INFO) { taosPrintLog("SYNC ", 255, __VA_ARGS__); }} while(0)
#define syncDebug(...) do { if (sDebugFlag & DEBUG_DEBUG) { taosPrintLog("SYNC ", sDebugFlag, __VA_ARGS__); }} while(0)
#define syncTrace(...) do { if (sDebugFlag & DEBUG_TRACE) { taosPrintLog("SYNC ", sDebugFlag, __VA_ARGS__); }} while(0)
#endif /* _TD_LIBS_SYNC_INT_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_CONST_H
#define _TD_LIBS_SYNC_CONST_H
#include "sync.h"
static int kSyncRaftMaxInflghtMsgs = 20;
static SyncIndex kMaxCommitIndex = UINT64_MAX;
#endif /* _TD_LIBS_SYNC_CONST_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_CONFIG_CHANGE_H
#define TD_SYNC_RAFT_CONFIG_CHANGE_H
#include "sync_type.h"
#include "sync_raft_proto.h"
/**
* Changer facilitates configuration changes. It exposes methods to handle
* simple and joint consensus while performing the proper validation that allows
* refusing invalid configuration changes before they affect the active
* configuration.
**/
struct SSyncRaftChanger {
SSyncRaftProgressTracker* tracker;
SyncIndex lastIndex;
};
typedef int (*configChangeFp)(SSyncRaftChanger* changer, const SSyncConfChangeSingleArray* css,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
// Simple carries out a series of configuration changes that (in aggregate)
// mutates the incoming majority config Voters[0] by at most one. This method
// will return an error if that is not the case, if the resulting quorum is
// zero, or if the configuration is in a joint state (i.e. if there is an
// outgoing configuration).
int syncRaftChangerSimpleConfig(SSyncRaftChanger* changer, const SSyncConfChangeSingleArray* css,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const SSyncConfChangeSingleArray* css,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
#endif /* TD_SYNC_RAFT_CONFIG_CHANGE_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_IMPL_H
#define _TD_LIBS_SYNC_RAFT_IMPL_H
#include "sync.h"
#include "sync_type.h"
#include "raft_message.h"
#include "sync_raft_quorum.h"
void syncRaftBecomeFollower(SSyncRaft* pRaft, SyncTerm term, SyncNodeId leaderId);
void syncRaftBecomePreCandidate(SSyncRaft* pRaft);
void syncRaftBecomeCandidate(SSyncRaft* pRaft);
void syncRaftBecomeLeader(SSyncRaft* pRaft);
void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType);
void syncRaftCampaign(SSyncRaft* pRaft, ESyncRaftElectionType cType);
void syncRaftTriggerHeartbeat(SSyncRaft* pRaft);
void syncRaftRandomizedElectionTimeout(SSyncRaft* pRaft);
bool syncRaftIsPromotable(SSyncRaft* pRaft);
bool syncRaftIsPastElectionTimeout(SSyncRaft* pRaft);
int syncRaftQuorum(SSyncRaft* pRaft);
bool syncRaftMaybeCommit(SSyncRaft* pRaft);
ESyncRaftVoteResult syncRaftPollVote(SSyncRaft* pRaft, SyncNodeId id,
bool preVote, bool accept,
int* rejectNum, int *granted);
static FORCE_INLINE bool syncRaftIsEmptyServerState(const SSyncServerState* serverState) {
return serverState->commitIndex == 0 &&
serverState->term == SYNC_NON_TERM &&
serverState->voteFor == SYNC_NON_NODE_ID;
}
void syncRaftLoadState(SSyncRaft* pRaft, const SSyncServerState* serverState);
void syncRaftBroadcastAppend(SSyncRaft* pRaft);
SNodeInfo* syncRaftGetNodeById(SSyncRaft *pRaft, SyncNodeId id);
#endif /* _TD_LIBS_SYNC_RAFT_IMPL_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http: *www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_INFLIGHTS_H
#define TD_SYNC_RAFT_INFLIGHTS_H
#include "sync.h"
// Inflights limits the number of MsgApp (represented by the largest index
// contained within) sent to followers but not yet acknowledged by them. Callers
// use Full() to check whether more messages can be sent, call Add() whenever
// they are sending a new append, and release "quota" via FreeLE() whenever an
// ack is received.
typedef struct SSyncRaftInflights {
// the starting index in the buffer
int start;
// number of inflights in the buffer
int count;
// the size of the buffer
int size;
// buffer contains the index of the last entry
// inside one message.
SyncIndex* buffer;
} SSyncRaftInflights;
SSyncRaftInflights* syncRaftOpenInflights(int size);
void syncRaftCloseInflights(SSyncRaftInflights*);
// reset frees all inflights.
static FORCE_INLINE void syncRaftInflightReset(SSyncRaftInflights* inflights) {
inflights->count = 0;
inflights->start = 0;
}
// Full returns true if no more messages can be sent at the moment.
static FORCE_INLINE bool syncRaftInflightFull(SSyncRaftInflights* inflights) {
return inflights->count == inflights->size;
}
// Add notifies the Inflights that a new message with the given index is being
// dispatched. Full() must be called prior to Add() to verify that there is room
// for one more message, and consecutive calls to add Add() must provide a
// monotonic sequence of indexes.
void syncRaftInflightAdd(SSyncRaftInflights* inflights, SyncIndex inflightIndex);
// FreeLE frees the inflights smaller or equal to the given `to` flight.
void syncRaftInflightFreeLE(SSyncRaftInflights* inflights, SyncIndex toIndex);
/**
* syncRaftInflightFreeFirstOne releases the first inflight.
* This is a no-op if nothing is inflight.
**/
void syncRaftInflightFreeFirstOne(SSyncRaftInflights* inflights);
#endif /* TD_SYNC_RAFT_INFLIGHTS_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_NODE_MAP_H
#define _TD_LIBS_SYNC_RAFT_NODE_MAP_H
#include "thash.h"
#include "sync.h"
#include "sync_type.h"
struct SSyncRaftNodeMap {
SHashObj* nodeIdMap;
};
void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap);
void syncRaftFreeNodeMap(SSyncRaftNodeMap* nodeMap);
void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap);
bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId);
void syncRaftCopyNodeMap(SSyncRaftNodeMap* from, SSyncRaftNodeMap* to);
void syncRaftUnionNodeMap(SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to);
void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId);
void syncRaftRemoveFromNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId);
int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap);
// return true if reach the end
bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId);
bool syncRaftIsAllNodeInProgressMap(SSyncRaftNodeMap* nodeMap, SSyncRaftProgressMap* progressMap);
#endif /* _TD_LIBS_SYNC_RAFT_NODE_MAP_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http: *www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_PROGRESS_H
#define TD_SYNC_RAFT_PROGRESS_H
#include "sync_type.h"
#include "sync_raft_inflights.h"
#include "thash.h"
/**
* State defines how the leader should interact with the follower.
*
* When in PROGRESS_STATE_PROBE, leader sends at most one replication message
* per heartbeat interval. It also probes actual progress of the follower.
*
* When in PROGRESS_STATE_REPLICATE, leader optimistically increases next
* to the latest entry sent after sending replication message. This is
* an optimized state for fast replicating log entries to the follower.
*
* When in PROGRESS_STATE_SNAPSHOT, leader should have sent out snapshot
* before and stops sending any replication message.
*
* PROGRESS_STATE_PROBE is the initial state.
**/
typedef enum ESyncRaftProgressState {
/**
* StateProbe indicates a follower whose last index isn't known. Such a
* follower is "probed" (i.e. an append sent periodically) to narrow down
* its last index. In the ideal (and common) case, only one round of probing
* is necessary as the follower will react with a hint. Followers that are
* probed over extended periods of time are often offline.
**/
PROGRESS_STATE_PROBE = 0,
/**
* StateReplicate is the state steady in which a follower eagerly receives
* log entries to append to its log.
**/
PROGRESS_STATE_REPLICATE,
/**
* StateSnapshot indicates a follower that needs log entries not available
* from the leader's Raft log. Such a follower needs a full snapshot to
* return to StateReplicate.
**/
PROGRESS_STATE_SNAPSHOT,
} ESyncRaftProgressState;
static const char* kProgressStateString[] = {
"Probe",
"Replicate",
"Snapshot",
};
// Progress represents a follower’s progress in the view of the leader. Leader
// maintains progresses of all followers, and sends entries to the follower
// based on its progress.
//
// NB(tbg): Progress is basically a state machine whose transitions are mostly
// strewn around `*raft.raft`. Additionally, some fields are only used when in a
// certain State. All of this isn't ideal.
struct SSyncRaftProgress {
SyncGroupId groupId;
SyncNodeId id;
int16_t refCount;
SyncIndex nextIndex;
SyncIndex matchIndex;
// State defines how the leader should interact with the follower.
//
// When in StateProbe, leader sends at most one replication message
// per heartbeat interval. It also probes actual progress of the follower.
//
// When in StateReplicate, leader optimistically increases next
// to the latest entry sent after sending replication message. This is
// an optimized state for fast replicating log entries to the follower.
//
// When in StateSnapshot, leader should have sent out snapshot
// before and stops sending any replication message.
ESyncRaftProgressState state;
// PendingSnapshot is used in StateSnapshot.
// If there is a pending snapshot, the pendingSnapshot will be set to the
// index of the snapshot. If pendingSnapshot is set, the replication process of
// this Progress will be paused. raft will not resend snapshot until the pending one
// is reported to be failed.
SyncIndex pendingSnapshotIndex;
// RecentActive is true if the progress is recently active. Receiving any messages
// from the corresponding follower indicates the progress is active.
// RecentActive can be reset to false after an election timeout.
//
// TODO(tbg): the leader should always have this set to true.
bool recentActive;
// ProbeSent is used while this follower is in StateProbe. When ProbeSent is
// true, raft should pause sending replication message to this peer until
// ProbeSent is reset. See ProbeAcked() and IsPaused().
bool probeSent;
// Inflights is a sliding window for the inflight messages.
// Each inflight message contains one or more log entries.
// The max number of entries per message is defined in raft config as MaxSizePerMsg.
// Thus inflight effectively limits both the number of inflight messages
// and the bandwidth each Progress can use.
// When inflights is Full, no more message should be sent.
// When a leader sends out a message, the index of the last
// entry should be added to inflights. The index MUST be added
// into inflights in order.
// When a leader receives a reply, the previous inflights should
// be freed by calling inflights.FreeLE with the index of the last
// received entry.
SSyncRaftInflights* inflights;
// IsLearner is true if this progress is tracked for a learner.
bool isLearner;
};
struct SSyncRaftProgressMap {
// map nodeId -> SSyncRaftProgress*
SHashObj* progressMap;
};
static FORCE_INLINE const char* syncRaftProgressStateString(const SSyncRaftProgress* progress) {
return kProgressStateString[progress->state];
}
void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress);
// BecomeProbe transitions into StateProbe. Next is reset to Match+1 or,
// optionally and if larger, the index of the pending snapshot.
void syncRaftProgressBecomeProbe(SSyncRaftProgress* progress);
// BecomeReplicate transitions into StateReplicate, resetting Next to Match+1.
void syncRaftProgressBecomeReplicate(SSyncRaftProgress* progress);
// MaybeUpdate is called when an MsgAppResp arrives from the follower, with the
// index acked by it. The method returns false if the given n index comes from
// an outdated message. Otherwise it updates the progress and returns true.
bool syncRaftProgressMaybeUpdate(SSyncRaftProgress* progress, SyncIndex lastIndex);
// OptimisticUpdate signals that appends all the way up to and including index n
// are in-flight. As a result, Next is increased to n+1.
static FORCE_INLINE void syncRaftProgressOptimisticNextIndex(SSyncRaftProgress* progress, SyncIndex nextIndex) {
progress->nextIndex = nextIndex + 1;
}
// MaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The
// arguments are the index of the append message rejected by the follower, and
// the hint that we want to decrease to.
//
// Rejections can happen spuriously as messages are sent out of order or
// duplicated. In such cases, the rejection pertains to an index that the
// Progress already knows were previously acknowledged, and false is returned
// without changing the Progress.
//
// If the rejection is genuine, Next is lowered sensibly, and the Progress is
// cleared for sending log entries.
bool syncRaftProgressMaybeDecrTo(SSyncRaftProgress* progress,
SyncIndex rejected, SyncIndex matchHint);
// IsPaused returns whether sending log entries to this node has been throttled.
// This is done when a node has rejected recent MsgApps, is currently waiting
// for a snapshot, or has reached the MaxInflightMsgs limit. In normal
// operation, this is false. A throttled node will be contacted less frequently
// until it has reached a state in which it's able to accept a steady stream of
// log entries again.
bool syncRaftProgressIsPaused(SSyncRaftProgress* progress);
static FORCE_INLINE SyncIndex syncRaftProgressNextIndex(SSyncRaftProgress* progress) {
return progress->nextIndex;
}
static FORCE_INLINE ESyncRaftProgressState syncRaftProgressInReplicate(SSyncRaftProgress* progress) {
return progress->state == PROGRESS_STATE_REPLICATE;
}
static FORCE_INLINE ESyncRaftProgressState syncRaftProgressInSnapshot(SSyncRaftProgress* progress) {
return progress->state == PROGRESS_STATE_SNAPSHOT;
}
static FORCE_INLINE ESyncRaftProgressState syncRaftProgressInProbe(SSyncRaftProgress* progress) {
return progress->state == PROGRESS_STATE_PROBE;
}
static FORCE_INLINE bool syncRaftProgressRecentActive(SSyncRaftProgress* progress) {
return progress->recentActive;
}
void syncRaftInitProgressMap(SSyncRaftProgressMap* progressMap);
void syncRaftFreeProgressMap(SSyncRaftProgressMap* progressMap);
void syncRaftClearProgressMap(SSyncRaftProgressMap* progressMap);
void syncRaftCopyProgressMap(SSyncRaftProgressMap* from, SSyncRaftProgressMap* to);
SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id);
int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress);
void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id);
bool syncRaftIsInProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id);
/**
* return true if progress's log is up-todate
**/
bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress);
// BecomeSnapshot moves the Progress to StateSnapshot with the specified pending
// snapshot index.
void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snapshotIndex);
void syncRaftCopyProgress(const SSyncRaftProgress* from, SSyncRaftProgress* to);
// return true if reach the end
bool syncRaftIterateProgressMap(const SSyncRaftProgressMap* progressMap, SSyncRaftProgress *pProgress);
bool syncRaftVisitProgressMap(SSyncRaftProgressMap* progressMap, visitProgressFp fp, void* arg);
#if 0
void syncRaftProgressAbortSnapshot(SSyncRaft* pRaft, int i);
SyncIndex syncRaftProgressMatchIndex(SSyncRaft* pRaft, int i);
void syncRaftProgressUpdateLastSend(SSyncRaft* pRaft, int i);
void syncRaftProgressUpdateSnapshotLastSend(SSyncRaft* pRaft, int i);
bool syncRaftProgressResetRecentRecv(SSyncRaft* pRaft, int i);
void syncRaftProgressMarkRecentRecv(SSyncRaft* pRaft, int i);
void syncRaftProgressAbortSnapshot(SSyncRaft* pRaft, int i);
#endif
#endif /* TD_SYNC_RAFT_PROGRESS_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_PROGRESS_TRACKER_H
#define _TD_LIBS_SYNC_RAFT_PROGRESS_TRACKER_H
#include "sync_type.h"
#include "sync_raft_quorum.h"
#include "sync_raft_quorum_joint.h"
#include "sync_raft_progress.h"
#include "sync_raft_proto.h"
#include "thash.h"
// Config reflects the configuration tracked in a ProgressTracker.
struct SSyncRaftProgressTrackerConfig {
SSyncRaftQuorumJointConfig voters;
// autoLeave is true if the configuration is joint and a transition to the
// incoming configuration should be carried out automatically by Raft when
// this is possible. If false, the configuration will be joint until the
// application initiates the transition manually.
bool autoLeave;
// Learners is a set of IDs corresponding to the learners active in the
// current configuration.
//
// Invariant: Learners and Voters does not intersect, i.e. if a peer is in
// either half of the joint config, it can't be a learner; if it is a
// learner it can't be in either half of the joint config. This invariant
// simplifies the implementation since it allows peers to have clarity about
// its current role without taking into account joint consensus.
SSyncRaftNodeMap learners;
// When we turn a voter into a learner during a joint consensus transition,
// we cannot add the learner directly when entering the joint state. This is
// because this would violate the invariant that the intersection of
// voters and learners is empty. For example, assume a Voter is removed and
// immediately re-added as a learner (or in other words, it is demoted):
//
// Initially, the configuration will be
//
// voters: {1 2 3}
// learners: {}
//
// and we want to demote 3. Entering the joint configuration, we naively get
//
// voters: {1 2} & {1 2 3}
// learners: {3}
//
// but this violates the invariant (3 is both voter and learner). Instead,
// we get
//
// voters: {1 2} & {1 2 3}
// learners: {}
// next_learners: {3}
//
// Where 3 is now still purely a voter, but we are remembering the intention
// to make it a learner upon transitioning into the final configuration:
//
// voters: {1 2}
// learners: {3}
// next_learners: {}
//
// Note that next_learners is not used while adding a learner that is not
// also a voter in the joint config. In this case, the learner is added
// right away when entering the joint configuration, so that it is caught up
// as soon as possible.
SSyncRaftNodeMap learnersNext;
};
struct SSyncRaftProgressTracker {
SSyncRaftProgressTrackerConfig config;
SSyncRaftProgressMap progressMap;
// nodeid -> ESyncRaftVoteType map
SHashObj* votesMap;
int maxInflightMsgs;
SSyncRaft* pRaft;
};
SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft);
void syncRaftInitTrackConfig(SSyncRaftProgressTrackerConfig* config);
void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config);
void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config);
// ResetVotes prepares for a new round of vote counting via recordVote.
void syncRaftResetVotes(SSyncRaftProgressTracker*);
void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, void* arg);
// RecordVote records that the node with the given id voted for this Raft
// instance if v == true (and declined it otherwise).
void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant);
void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to);
int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
// TallyVotes returns the number of granted and rejected Votes, and whether the
// election outcome is known.
ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* rejected, int *granted);
void syncRaftConfigState(SSyncRaftProgressTracker* tracker, SSyncConfigState* cs);
// Committed returns the largest log index known to be committed based on what
// the voting members of the group have acknowledged.
SyncIndex syncRaftCommittedIndex(SSyncRaftProgressTracker* tracker);
// QuorumActive returns true if the quorum is active from the view of the local
// raft state machine. Otherwise, it returns false.
bool syncRaftQuorumActive(SSyncRaftProgressTracker* tracker);
bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId);
#endif /* _TD_LIBS_SYNC_RAFT_PROGRESS_TRACKER_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_PROTO_H
#define TD_SYNC_RAFT_PROTO_H
#include "sync_type.h"
#include "sync_raft_node_map.h"
typedef enum ESyncRaftConfChangeType {
SYNC_RAFT_Conf_AddNode = 0,
SYNC_RAFT_Conf_RemoveNode = 1,
SYNC_RAFT_Conf_UpdateNode = 2,
SYNC_RAFT_Conf_AddLearnerNode = 3,
} ESyncRaftConfChangeType;
// ConfChangeSingle is an individual configuration change operation. Multiple
// such operations can be carried out atomically via a ConfChangeV2.
typedef struct SSyncConfChangeSingle {
ESyncRaftConfChangeType type;
SyncNodeId nodeId;
} SSyncConfChangeSingle;
typedef struct SSyncConfChangeSingleArray {
int n;
SSyncConfChangeSingle* changes;
} SSyncConfChangeSingleArray;
typedef struct SSyncConfigState {
// The voters in the incoming config. (If the configuration is not joint,
// then the outgoing config is empty).
SSyncRaftNodeMap voters;
// The learners in the incoming config.
SSyncRaftNodeMap learners;
// The voters in the outgoing config.
SSyncRaftNodeMap votersOutgoing;
// The nodes that will become learners when the outgoing config is removed.
// These nodes are necessarily currently in nodes_joint (or they would have
// been added to the incoming config right away).
SSyncRaftNodeMap learnersNext;
// If set, the config is joint and Raft will automatically transition into
// the final config (i.e. remove the outgoing config) when this is safe.
bool autoLeave;
} SSyncConfigState;
static FORCE_INLINE bool syncRaftConfArrayIsEmpty(const SSyncConfChangeSingleArray* ary) {
return ary->n == 0;
}
static FORCE_INLINE void syncRaftInitConfArray(SSyncConfChangeSingleArray* ary) {
*ary = (SSyncConfChangeSingleArray) {
.changes = NULL,
.n = 0,
};
}
static FORCE_INLINE void syncRaftFreeConfArray(SSyncConfChangeSingleArray* ary) {
if (ary->changes != NULL) free(ary->changes);
}
#endif /* TD_SYNC_RAFT_PROTO_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_QUORUM_H
#define TD_SYNC_RAFT_QUORUM_H
/**
* ESyncRaftVoteResult indicates the outcome of a vote.
**/
typedef enum {
/**
* SYNC_RAFT_VOTE_PENDING indicates that the decision of the vote depends on future
* votes, i.e. neither "yes" or "no" has reached quorum yet.
**/
SYNC_RAFT_VOTE_PENDING = 1,
/**
* SYNC_RAFT_VOTE_LOST indicates that the quorum has voted "no".
**/
SYNC_RAFT_VOTE_LOST = 2,
/**
* SYNC_RAFT_VOTE_WON indicates that the quorum has voted "yes".
**/
SYNC_RAFT_VOTE_WON = 3,
} ESyncRaftVoteResult;
#endif /* TD_SYNC_RAFT_QUORUM_H */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_QUORUM_JOINT_H
#define _TD_LIBS_SYNC_RAFT_QUORUM_JOINT_H
#include "taosdef.h"
#include "sync.h"
#include "sync_type.h"
#include "sync_raft_node_map.h"
#include "thash.h"
// JointConfig is a configuration of two groups of (possibly overlapping)
// majority configurations. Decisions require the support of both majorities.
typedef struct SSyncRaftQuorumJointConfig {
SSyncRaftNodeMap outgoing;
SSyncRaftNodeMap incoming;
} SSyncRaftQuorumJointConfig;
// IDs returns a newly initialized map representing the set of voters present
// in the joint configuration.
void syncRaftJointConfigIDs(SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap);
// CommittedIndex returns the largest committed index for the given joint
// quorum. An index is jointly committed if it is committed in both constituent
// majorities.
SyncIndex syncRaftJointConfigCommittedIndex(const SSyncRaftQuorumJointConfig* config, matchAckIndexerFp indexer, void* arg);
// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns
// a result indicating whether the vote is pending, lost, or won. A joint quorum
// requires both majority quorums to vote in favor.
ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap);
void syncRaftInitQuorumJointConfig(SSyncRaftQuorumJointConfig* config);
static FORCE_INLINE bool syncRaftJointConfigInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) {
return syncRaftIsInNodeMap(&config->outgoing, id);
}
static FORCE_INLINE bool syncRaftJointConfigInIncoming(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) {
return syncRaftIsInNodeMap(&config->incoming, id);
}
void syncRaftJointConfigAddToIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id);
void syncRaftJointConfigRemoveFromIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id);
static FORCE_INLINE const SSyncRaftNodeMap* syncRaftJointConfigIncoming(const SSyncRaftQuorumJointConfig* config) {
return &config->incoming;
}
static FORCE_INLINE const SSyncRaftNodeMap* syncRaftJointConfigOutgoing(const SSyncRaftQuorumJointConfig* config) {
return &config->outgoing;
}
static FORCE_INLINE void syncRaftJointConfigClearOutgoing(SSyncRaftQuorumJointConfig* config) {
syncRaftClearNodeMap(&config->outgoing);
}
static FORCE_INLINE bool syncRaftJointConfigIsIncomingEmpty(const SSyncRaftQuorumJointConfig* config) {
return syncRaftNodeMapSize(&config->incoming) == 0;
}
static FORCE_INLINE bool syncRaftJointConfigIsOutgoingEmpty(const SSyncRaftQuorumJointConfig* config) {
return syncRaftNodeMapSize(&config->outgoing) == 0;
}
static FORCE_INLINE bool syncRaftJointConfigIsInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) {
return syncRaftIsInNodeMap(&config->outgoing, id);
}
#endif /* _TD_LIBS_SYNC_RAFT_QUORUM_JOINT_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_RAFT_QUORUM_MAJORITY_H
#define _TD_LIBS_SYNC_RAFT_QUORUM_MAJORITY_H
#include "sync.h"
#include "sync_type.h"
#include "sync_raft_quorum.h"
#include "thash.h"
/**
* syncRaftMajorityVoteResult takes a mapping of voters to yes/no (true/false) votes and returns
* a result indicating whether the vote is pending (i.e. neither a quorum of
* yes/no has been reached), won (a quorum of yes has been reached), or lost (a
* quorum of no has been reached).
**/
ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap);
// CommittedIndex computes the committed index from those supplied via the
// provided AckedIndexer (for the active config).
SyncIndex syncRaftMajorityConfigCommittedIndex(const SSyncRaftNodeMap* config, matchAckIndexerFp indexer, void* arg);
#endif /* _TD_LIBS_SYNC_RAFT_QUORUM_MAJORITY_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TD_SYNC_RAFT_RESTORE_H
#define TD_SYNC_RAFT_RESTORE_H
#include "sync_type.h"
#include "sync_raft_proto.h"
// syncRaftRestoreConfig takes a Changer (which must represent an empty configuration), and
// runs a sequence of changes enacting the configuration described in the
// ConfState.
//
// TODO(tbg) it's silly that this takes a Changer. Unravel this by making sure
// the Changer only needs a ProgressMap (not a whole Tracker) at which point
// this can just take LastIndex and MaxInflight directly instead and cook up
// the results from that alone.
int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
#endif /* TD_SYNC_RAFT_RESTORE_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_LIBS_SYNC_TYPE_H
#define _TD_LIBS_SYNC_TYPE_H
#include <stdint.h>
#include "sync.h"
#include "osMath.h"
#define SYNC_NON_NODE_ID -1
#define SYNC_NON_TERM 0
typedef int32_t SyncTime;
typedef uint32_t SyncTick;
typedef struct SSyncRaft SSyncRaft;
typedef struct SSyncRaftProgress SSyncRaftProgress;
typedef struct SSyncRaftProgressMap SSyncRaftProgressMap;
typedef struct SSyncRaftProgressTrackerConfig SSyncRaftProgressTrackerConfig;
typedef struct SSyncRaftNodeMap SSyncRaftNodeMap;
typedef struct SSyncRaftProgressTracker SSyncRaftProgressTracker;
typedef struct SSyncRaftChanger SSyncRaftChanger;
typedef struct SSyncRaftLog SSyncRaftLog;
typedef struct SSyncRaftEntry SSyncRaftEntry;
#if 0
#ifndef TMIN
#define TMIN(x, y) (((x) < (y)) ? (x) : (y))
#endif
#ifndef TMAX
#define TMAX(x, y) (((x) > (y)) ? (x) : (y))
#endif
#endif
typedef struct SSyncServerState {
SyncNodeId voteFor;
SyncTerm term;
SyncIndex commitIndex;
} SSyncServerState;
typedef struct SSyncClusterConfig {
// Log index number of current cluster config.
SyncIndex index;
// Log index number of previous cluster config.
SyncIndex prevIndex;
// current cluster
const SSyncCluster* cluster;
} SSyncClusterConfig;
typedef enum {
SYNC_RAFT_CAMPAIGN_PRE_ELECTION = 0,
SYNC_RAFT_CAMPAIGN_ELECTION = 1,
SYNC_RAFT_CAMPAIGN_TRANSFER = 2,
} ESyncRaftElectionType;
typedef enum {
// grant the vote request
SYNC_RAFT_VOTE_RESP_GRANT = 1,
// reject the vote request
SYNC_RAFT_VOTE_RESP_REJECT = 2,
} ESyncRaftVoteType;
typedef void (*visitProgressFp)(SSyncRaftProgress* progress, void* arg);
typedef void (*matchAckIndexerFp)(SyncNodeId id, void* arg, SyncIndex* index);
#endif /* _TD_LIBS_SYNC_TYPE_H */
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "sync_raft_impl.h"
#include "raft_log.h"
#include "sync_raft_restore.h"
#include "raft_replication.h"
#include "sync_raft_config_change.h"
#include "sync_raft_progress_tracker.h"
#include "syncInt.h"
#define RAFT_READ_LOG_MAX_NUM 100
static int deserializeServerStateFromBuffer(SSyncServerState* server, const char* buffer, int n);
static int deserializeClusterStateFromBuffer(SSyncConfigState* cluster, const char* buffer, int n);
static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfig* config,
const SSyncRaftProgressMap* progressMap, SSyncConfigState* cs);
static void abortLeaderTransfer(SSyncRaft* pRaft);
static bool preHandleMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
static bool preHandleNewTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
static bool preHandleOldTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int32_t syncRaftStart(SSyncRaft* pRaft, const SSyncInfo* pInfo) {
SSyncNode* pNode = pRaft->pNode;
SSyncServerState serverState;
SSyncConfigState confState;
SStateManager* stateManager;
SSyncLogStore* logStore;
SSyncFSM* fsm;
SSyncBuffer buffer[RAFT_READ_LOG_MAX_NUM];
int nBuf, limit, i;
char* buf;
int n;
SSyncRaftChanger changer;
memset(pRaft, 0, sizeof(SSyncRaft));
memcpy(&pRaft->fsm, &pInfo->fsm, sizeof(SSyncFSM));
memcpy(&pRaft->logStore, &pInfo->logStore, sizeof(SSyncLogStore));
memcpy(&pRaft->stateManager, &pInfo->stateManager, sizeof(SStateManager));
stateManager = &(pRaft->stateManager);
logStore = &(pRaft->logStore);
fsm = &(pRaft->fsm);
pRaft->nodeInfoMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
if (pRaft->nodeInfoMap == NULL) {
return -1;
}
// init progress tracker
pRaft->tracker = syncRaftOpenProgressTracker(pRaft);
if (pRaft->tracker == NULL) {
return -1;
}
// open raft log
if ((pRaft->log = syncRaftLogOpen()) == NULL) {
return -1;
}
// read server state
if (stateManager->readServerState(stateManager, &buf, &n) != 0) {
syncError("readServerState for vgid %d fail", pInfo->vgId);
return -1;
}
if (deserializeServerStateFromBuffer(&serverState, buf, n) != 0) {
syncError("deserializeServerStateFromBuffer for vgid %d fail", pInfo->vgId);
return -1;
}
free(buf);
//assert(initIndex <= serverState.commitIndex);
// read config state
if (stateManager->readClusterState(stateManager, &buf, &n) != 0) {
syncError("readClusterState for vgid %d fail", pInfo->vgId);
return -1;
}
if (deserializeClusterStateFromBuffer(&confState, buf, n) != 0) {
syncError("deserializeClusterStateFromBuffer for vgid %d fail", pInfo->vgId);
return -1;
}
free(buf);
changer = (SSyncRaftChanger) {
.tracker = pRaft->tracker,
.lastIndex = syncRaftLogLastIndex(pRaft->log),
};
SSyncRaftProgressTrackerConfig config;
SSyncRaftProgressMap progressMap;
if (syncRaftRestoreConfig(&changer, &confState, &config, &progressMap) < 0) {
syncError("syncRaftRestoreConfig for vgid %d fail", pInfo->vgId);
return -1;
}
// save restored config and progress map to tracker
syncRaftCopyProgressMap(&progressMap, &pRaft->tracker->progressMap);
syncRaftCopyTrackerConfig(&config, &pRaft->tracker->config);
// free progress map and config
syncRaftFreeProgressMap(&progressMap);
syncRaftFreeTrackConfig(&config);
if (!syncRaftIsEmptyServerState(&serverState)) {
syncRaftLoadState(pRaft, &serverState);
}
if (pInfo->appliedIndex > 0) {
syncRaftLogAppliedTo(pRaft->log, pInfo->appliedIndex);
}
syncRaftBecomeFollower(pRaft, pRaft->term, SYNC_NON_NODE_ID);
syncInfo("[%d:%d] restore vgid %d state: snapshot index success",
pRaft->selfGroupId, pRaft->selfId, pInfo->vgId);
return 0;
}
int32_t syncRaftStep(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
syncDebug("from %d, type:%d, term:%" PRId64 ", state:%d",
pMsg->from, pMsg->msgType, pMsg->term, pRaft->state);
if (preHandleMessage(pRaft, pMsg)) {
syncFreeMessage(pMsg);
return 0;
}
ESyncRaftMessageType msgType = pMsg->msgType;
if (msgType == RAFT_MSG_INTERNAL_ELECTION) {
syncRaftHandleElectionMessage(pRaft, pMsg);
} else if (msgType == RAFT_MSG_VOTE) {
syncRaftHandleVoteMessage(pRaft, pMsg);
} else {
pRaft->stepFp(pRaft, pMsg);
}
syncFreeMessage(pMsg);
return 0;
}
int32_t syncRaftTick(SSyncRaft* pRaft) {
pRaft->currentTick += 1;
pRaft->tickFp(pRaft);
return 0;
}
static int deserializeServerStateFromBuffer(SSyncServerState* server, const char* buffer, int n) {
return 0;
}
static int deserializeClusterStateFromBuffer(SSyncConfigState* cluster, const char* buffer, int n) {
return 0;
}
static void visitProgressMaybeSendAppend(SSyncRaftProgress* progress, void* arg) {
syncRaftMaybeSendAppend(arg, progress, false);
}
// switchToConfig reconfigures this node to use the provided configuration. It
// updates the in-memory state and, when necessary, carries out additional
// actions such as reacting to the removal of nodes or changed quorum
// requirements.
//
// The inputs usually result from restoring a ConfState or applying a ConfChange.
static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfig* config,
const SSyncRaftProgressMap* progressMap, SSyncConfigState* cs) {
SyncNodeId selfId = pRaft->selfId;
int i;
bool exist;
SSyncRaftProgress* progress = NULL;
syncRaftConfigState(pRaft->tracker, cs);
progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, selfId);
exist = (progress != NULL);
// Update whether the node itself is a learner, resetting to false when the
// node is removed.
if (exist) {
pRaft->isLearner = progress->isLearner;
} else {
pRaft->isLearner = false;
}
if ((!exist || pRaft->isLearner) && pRaft->state == TAOS_SYNC_STATE_LEADER) {
// This node is leader and was removed or demoted. We prevent demotions
// at the time writing but hypothetically we handle them the same way as
// removing the leader: stepping down into the next Term.
//
// TODO(tbg): step down (for sanity) and ask follower with largest Match
// to TimeoutNow (to avoid interruption). This might still drop some
// proposals but it's better than nothing.
//
// TODO(tbg): test this branch. It is untested at the time of writing.
return;
}
// The remaining steps only make sense if this node is the leader and there
// are other nodes.
if (pRaft->state != TAOS_SYNC_STATE_LEADER || syncRaftNodeMapSize(&cs->voters) == 0) {
return;
}
if (syncRaftMaybeCommit(pRaft)) {
// If the configuration change means that more entries are committed now,
// broadcast/append to everyone in the updated config.
syncRaftBroadcastAppend(pRaft);
} else {
// Otherwise, still probe the newly added replicas; there's no reason to
// let them wait out a heartbeat interval (or the next incoming
// proposal).
syncRaftProgressVisit(pRaft->tracker, visitProgressMaybeSendAppend, pRaft);
// If the the leadTransferee was removed or demoted, abort the leadership transfer.
SyncNodeId leadTransferee = pRaft->leadTransferee;
if (leadTransferee != SYNC_NON_NODE_ID) {
if (!syncRaftIsInNodeMap(&pRaft->tracker->config.voters.incoming, leadTransferee) &&
!syncRaftIsInNodeMap(&pRaft->tracker->config.voters.outgoing, leadTransferee)) {
abortLeaderTransfer(pRaft);
}
}
}
}
static void abortLeaderTransfer(SSyncRaft* pRaft) {
pRaft->leadTransferee = SYNC_NON_NODE_ID;
}
/**
* pre-handle message, return true means no need to continue
* Handle the message term, which may result in our stepping down to a follower.
**/
static bool preHandleMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
// local message?
if (pMsg->term == 0) {
return false;
}
if (pMsg->term > pRaft->term) {
return preHandleNewTermMessage(pRaft, pMsg);
} else if (pMsg->term < pRaft->term) {
return preHandleOldTermMessage(pRaft, pMsg);
}
return false;
}
static bool preHandleNewTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
SyncNodeId leaderId = pMsg->from;
ESyncRaftMessageType msgType = pMsg->msgType;
if (msgType == RAFT_MSG_VOTE) {
// TODO
leaderId = SYNC_NON_NODE_ID;
}
if (syncIsPreVoteMsg(pMsg)) {
// Never change our term in response to a PreVote
} else if (syncIsPreVoteRespMsg(pMsg) && !pMsg->voteResp.rejected) {
/**
* We send pre-vote requests with a term in our future. If the
* pre-vote is granted, we will increment our term when we get a
* quorum. If it is not, the term comes from the node that
* rejected our vote so we should become a follower at the new
* term.
**/
} else {
syncInfo("[%d:%d] [term:%" PRId64 "] received a %d message with higher term from %d [term:%" PRId64 "]",
pRaft->selfGroupId, pRaft->selfId, pRaft->term, msgType, pMsg->from, pMsg->term);
syncRaftBecomeFollower(pRaft, pMsg->term, leaderId);
}
return false;
}
static bool preHandleOldTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
if (pRaft->checkQuorum && pMsg->msgType == RAFT_MSG_APPEND) {
/**
* We have received messages from a leader at a lower term. It is possible
* that these messages were simply delayed in the network, but this could
* also mean that this node has advanced its term number during a network
* partition, and it is now unable to either win an election or to rejoin
* the majority on the old term. If checkQuorum is false, this will be
* handled by incrementing term numbers in response to MsgVote with a
* higher term, but if checkQuorum is true we may not advance the term on
* MsgVote and must generate other messages to advance the term. The net
* result of these two features is to minimize the disruption caused by
* nodes that have been removed from the cluster's configuration: a
* removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
* but it will not receive MsgApp or MsgHeartbeat, so it will not create
* disruptive term increases
**/
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from);
if (pNode == NULL) {
return true;
}
SSyncMessage* msg = syncNewEmptyAppendRespMsg(pRaft->selfGroupId, pRaft->selfId, pRaft->term);
if (msg == NULL) {
return true;
}
pRaft->io.send(msg, pNode);
} else {
// ignore other cases
syncInfo("[%d:%d] [term:%" PRId64 "] ignored a %d message with lower term from %d [term:%" PRId64 "]",
pRaft->selfGroupId, pRaft->selfId, pRaft->term, pMsg->msgType, pMsg->from, pMsg->term);
}
return true;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "raft.h"
#include "raft_log.h"
#include "sync_raft_impl.h"
#include "raft_message.h"
int syncRaftHandleAppendEntriesMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
const RaftMsg_Append_Entries *appendEntries = &(pMsg->appendEntries);
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from);
if (pNode == NULL) {
return 0;
}
SSyncMessage* pRespMsg = syncNewEmptyAppendRespMsg(pRaft->selfGroupId, pRaft->selfId, pRaft->term);
if (pRespMsg == NULL) {
return 0;
}
RaftMsg_Append_Resp *appendResp = &(pRespMsg->appendResp);
// ignore committed logs
if (syncRaftLogIsCommitted(pRaft->log, appendEntries->index)) {
appendResp->index = pRaft->log->commitIndex;
goto out;
}
syncInfo("[%d:%d] recv append from %d index %" PRId64"",
pRaft->selfGroupId, pRaft->selfId, pMsg->from, appendEntries->index);
out:
pRaft->io.send(pRespMsg, pNode);
return 0;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "raft.h"
#include "raft_log.h"
#include "raft_message.h"
int syncRaftHandleElectionMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
if (pRaft->preVote) {
syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_PRE_ELECTION);
} else {
syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION);
}
return 0;
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "raft.h"
#include "sync_raft_impl.h"
#include "raft_log.h"
#include "raft_message.h"
static bool canGrantVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg);
int syncRaftHandleVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
SSyncMessage* pRespMsg;
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from);
if (pNode == NULL) {
return 0;
}
bool grant;
SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log);
SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log);
grant = canGrantVoteMessage(pRaft, pMsg);
pRespMsg = syncNewVoteRespMsg(pRaft->selfGroupId, pRaft->selfId, pMsg->vote.cType, !grant);
if (pRespMsg == NULL) {
return 0;
}
syncInfo("[%d:%d] [logterm: %" PRId64 ", index: %" PRId64 ", vote: %d] %s for %d"
"[logterm: %" PRId64 ", index: %" PRId64 "] at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, lastTerm, lastIndex, pRaft->voteFor,
grant ? "grant" : "reject",
pMsg->from, pMsg->vote.lastTerm, pMsg->vote.lastIndex, pRaft->term);
pRaft->io.send(pRespMsg, pNode);
return 0;
}
static bool canGrantVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
bool canVote =
// We can vote if this is a repeat of a vote we've already cast...
pRaft->voteFor == pMsg->from ||
// ...we haven't voted and we don't think there's a leader yet in this term...
(pRaft->voteFor == SYNC_NON_NODE_ID && pRaft->leaderId == SYNC_NON_NODE_ID) ||
// ...or this is a PreVote for a future term...
(pMsg->vote.cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION && pMsg->term > pRaft->term);
// ...and we believe the candidate is up to date.
return canVote && syncRaftLogIsUptodate(pRaft->log, pMsg->vote.lastIndex, pMsg->vote.lastTerm);
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "raft.h"
#include "sync_raft_impl.h"
#include "raft_message.h"
int syncRaftHandleVoteRespMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
int granted, rejected;
int quorum;
int voterIndex;
assert(pRaft->state == TAOS_SYNC_STATE_CANDIDATE);
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from);
if (pNode == NULL) {
syncError("[%d:%d] recv vote resp from unknown server %d", pRaft->selfGroupId, pRaft->selfId, pMsg->from);
return 0;
}
if (pRaft->state != TAOS_SYNC_STATE_CANDIDATE) {
syncError("[%d:%d] is not candidate, ignore vote resp", pRaft->selfGroupId, pRaft->selfId);
return 0;
}
ESyncRaftVoteResult result = syncRaftPollVote(pRaft, pMsg->from,
pMsg->voteResp.cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION,
!pMsg->voteResp.rejected, &rejected, &granted);
syncInfo("[%d:%d] [quorum:%d] has received %d votes and %d vote rejections",
pRaft->selfGroupId, pRaft->selfId, quorum, granted, rejected);
if (result == SYNC_RAFT_VOTE_WON) {
if (pRaft->candidateState.inPreVote) {
syncRaftCampaign(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION);
} else {
syncRaftBecomeLeader(pRaft);
syncRaftBroadcastAppend(pRaft);
}
} else if (result == SYNC_RAFT_VOTE_LOST) {
// pb.MsgPreVoteResp contains future term of pre-candidate
// m.Term > r.Term; reuse r.Term
syncRaftBecomeFollower(pRaft, pRaft->term, SYNC_NON_NODE_ID);
}
return 0;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft_log.h"
SSyncRaftLog* syncRaftLogOpen() {
return NULL;
}
SyncIndex syncRaftLogLastIndex(SSyncRaftLog* pLog) {
return 0;
}
SyncIndex syncRaftLogSnapshotIndex(SSyncRaftLog* pLog) {
return 0;
}
SyncTerm syncRaftLogLastTerm(SSyncRaftLog* pLog) {
return 0;
}
void syncRaftLogAppliedTo(SSyncRaftLog* pLog, SyncIndex appliedIndex) {
}
bool syncRaftLogIsUptodate(SSyncRaftLog* pLog, SyncIndex index, SyncTerm term) {
return true;
}
int syncRaftLogNumOfPendingConf(SSyncRaftLog* pLog) {
return 0;
}
bool syncRaftHasUnappliedLog(SSyncRaftLog* pLog) {
return pLog->commitIndex > pLog->appliedIndex;
}
SyncTerm syncRaftLogTermOf(SSyncRaftLog* pLog, SyncIndex index) {
return SYNC_NON_TERM;
}
int syncRaftLogAppend(SSyncRaftLog* pLog, SSyncRaftEntry *pEntries, int n) {
}
int syncRaftLogAcquire(SSyncRaftLog* pLog, SyncIndex index, int maxMsgSize,
SSyncRaftEntry **ppEntries, int *n) {
return 0;
}
void syncRaftLogRelease(SSyncRaftLog* pLog, SyncIndex index,
SSyncRaftEntry *pEntries, int n) {
return;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft_message.h"
void syncFreeMessage(const SSyncMessage* pMsg) {
if (!syncIsInternalMsg(pMsg->msgType)) {
free((SSyncMessage*)pMsg);
}
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "raft_log.h"
#include "sync_raft_progress.h"
#include "syncInt.h"
#include "raft_replication.h"
static bool sendSnapshot(SSyncRaft* pRaft, SSyncRaftProgress* progress);
static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress,
SyncIndex prevIndex, SyncTerm prevTerm,
SSyncRaftEntry *entries, int nEntry);
// maybeSendAppend sends an append RPC with new entries to the given peer,
// if necessary. Returns true if a message was sent. The sendIfEmpty
// argument controls whether messages with no entries will be sent
// ("empty" messages are useful to convey updated Commit indexes, but
// are undesirable when we're sending multiple messages in a batch).
bool syncRaftMaybeSendAppend(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty) {
assert(pRaft->state == TAOS_SYNC_STATE_LEADER);
SyncNodeId nodeId = progress->id;
if (syncRaftProgressIsPaused(progress)) {
syncInfo("node [%d:%d] paused", pRaft->selfGroupId, nodeId);
return false;
}
SyncIndex nextIndex = syncRaftProgressNextIndex(progress);
SSyncRaftEntry *entries;
int nEntry;
SyncIndex prevIndex;
SyncTerm prevTerm;
prevIndex = nextIndex - 1;
prevTerm = syncRaftLogTermOf(pRaft->log, prevIndex);
int ret = syncRaftLogAcquire(pRaft->log, nextIndex, pRaft->maxMsgSize, &entries, &nEntry);
if (nEntry == 0 && !sendIfEmpty) {
return false;
}
if (ret != 0 || prevTerm == SYNC_NON_TERM) {
return sendSnapshot(pRaft, progress);
}
return sendAppendEntries(pRaft, progress, prevIndex, prevTerm, entries, nEntry);
}
static bool sendSnapshot(SSyncRaft* pRaft, SSyncRaftProgress* progress) {
if (!syncRaftProgressRecentActive(progress)) {
return false;
}
return true;
}
static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress,
SyncIndex prevIndex, SyncTerm prevTerm,
SSyncRaftEntry *entries, int nEntry) {
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, progress->id);
if (pNode == NULL) {
return false;
}
SyncIndex lastIndex;
SyncTerm logTerm = prevTerm;
SSyncMessage* msg = syncNewAppendMsg(pRaft->selfGroupId, pRaft->selfId, pRaft->term,
prevIndex, prevTerm, pRaft->log->commitIndex,
nEntry, entries);
if (msg == NULL) {
goto err_release_log;
}
if (nEntry != 0) {
switch (progress->state) {
// optimistically increase the next when in StateReplicate
case PROGRESS_STATE_REPLICATE:
lastIndex = entries[nEntry - 1].index;
syncRaftProgressOptimisticNextIndex(progress, lastIndex);
syncRaftInflightAdd(progress->inflights, lastIndex);
break;
case PROGRESS_STATE_PROBE:
progress->probeSent = true;
break;
default:
syncFatal("[%d:%d] is sending append in unhandled state %s",
pRaft->selfGroupId, pRaft->selfId, syncRaftProgressStateString(progress));
break;
}
}
pRaft->io.send(msg, pNode);
return true;
err_release_log:
syncRaftLogRelease(pRaft->log, prevIndex + 1, entries, nEntry);
return false;
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "sync.h"
#include "raft_unstable_log.h"
/*
SyncIndex syncRaftLogLastIndex(SSyncRaftUnstableLog* pLog) {
return 0;
}
*/
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "trpc.h"
#include "ttimer.h"
SSyncManager* gSyncManager = NULL;
#define SYNC_TICK_TIMER 50
#define SYNC_ACTIVITY_TIMER 5
#define SYNC_SERVER_WORKER 2
static void syncProcessRsp(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet);
static void syncProcessReqMsg(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet);
static int syncInitRpcServer(SSyncManager* syncManager, const SSyncCluster* pSyncCfg);
static int syncInitRpcClient(SSyncManager* syncManager);
static int syncOpenWorkerPool(SSyncManager* syncManager);
static int syncCloseWorkerPool(SSyncManager* syncManager);
static void *syncWorkerMain(void *argv);
static void syncNodeTick(void *param, void *tmrId);
int32_t syncInit() {
if (gSyncManager != NULL) {
return 0;
}
gSyncManager = (SSyncManager*)calloc(sizeof(SSyncManager), 0);
if (gSyncManager == NULL) {
syncError("malloc SSyncManager fail");
return -1;
}
pthread_mutex_init(&gSyncManager->mutex, NULL);
// init client rpc
if (syncInitRpcClient(gSyncManager) != 0) {
syncCleanUp();
return -1;
}
// init sync timer manager
gSyncManager->syncTimerManager = taosTmrInit(1000, 50, 10000, "SYNC");
if (gSyncManager->syncTimerManager == NULL) {
syncCleanUp();
return -1;
}
// init worker pool
if (syncOpenWorkerPool(gSyncManager) != 0) {
syncCleanUp();
return -1;
}
// init vgroup hash table
gSyncManager->vgroupTable = taosHashInit(TSDB_MIN_VNODES, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
if (gSyncManager->vgroupTable == NULL) {
syncCleanUp();
return -1;
}
return 0;
}
void syncCleanUp() {
if (gSyncManager == NULL) {
return;
}
pthread_mutex_lock(&gSyncManager->mutex);
if (gSyncManager->vgroupTable) {
taosHashCleanup(gSyncManager->vgroupTable);
}
if (gSyncManager->clientRpc) {
rpcClose(gSyncManager->clientRpc);
syncInfo("sync inter-sync rpc client is closed");
}
if (gSyncManager->syncTimerManager) {
taosTmrCleanUp(gSyncManager->syncTimerManager);
}
syncCloseWorkerPool(gSyncManager);
pthread_mutex_unlock(&gSyncManager->mutex);
pthread_mutex_destroy(&gSyncManager->mutex);
free(gSyncManager);
gSyncManager = NULL;
}
SSyncNode* syncStart(const SSyncInfo* pInfo) {
pthread_mutex_lock(&gSyncManager->mutex);
SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pInfo->vgId, sizeof(SyncGroupId*));
if (ppNode != NULL) {
syncInfo("vgroup %d already exist", pInfo->vgId);
pthread_mutex_unlock(&gSyncManager->mutex);
return *ppNode;
}
// init rpc server
if (syncInitRpcServer(gSyncManager, &pInfo->syncCfg) != 0) {
pthread_mutex_unlock(&gSyncManager->mutex);
return NULL;
}
SSyncNode *pNode = (SSyncNode*)malloc(sizeof(SSyncNode));
if (pNode == NULL) {
syncError("malloc vgroup %d node fail", pInfo->vgId);
pthread_mutex_unlock(&gSyncManager->mutex);
return NULL;
}
pNode->syncTimer = taosTmrStart(syncNodeTick, SYNC_TICK_TIMER, (void*)((int64_t)pInfo->vgId), gSyncManager->syncTimerManager);
// start raft
pNode->raft.pNode = pNode;
if (syncRaftStart(&pNode->raft, pInfo) != 0) {
syncError("raft start at %d node fail", pInfo->vgId);
pthread_mutex_unlock(&gSyncManager->mutex);
return NULL;
}
pthread_mutex_init(&pNode->mutex, NULL);
taosHashPut(gSyncManager->vgroupTable, &pInfo->vgId, sizeof(SyncGroupId), &pNode, sizeof(SSyncNode *));
pthread_mutex_unlock(&gSyncManager->mutex);
return NULL;
}
void syncStop(const SSyncNode* pNode) {
pthread_mutex_lock(&gSyncManager->mutex);
SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pNode->vgId, sizeof(SyncGroupId*));
if (ppNode == NULL) {
syncInfo("vgroup %d not exist", pNode->vgId);
pthread_mutex_unlock(&gSyncManager->mutex);
return;
}
assert(*ppNode == pNode);
taosTmrStop(pNode->syncTimer);
taosHashRemove(gSyncManager->vgroupTable, &pNode->vgId, sizeof(SyncGroupId));
pthread_mutex_unlock(&gSyncManager->mutex);
pthread_mutex_destroy(&((*ppNode)->mutex));
free(*ppNode);
}
int32_t syncPropose(SSyncNode* syncNode, const SSyncBuffer* pBuf, void* pData, bool isWeak) {
SSyncMessage msg;
pthread_mutex_lock(&syncNode->mutex);
int32_t ret = syncRaftStep(&syncNode->raft, syncInitPropMsg(&msg, pBuf, pData, isWeak));
pthread_mutex_unlock(&syncNode->mutex);
return ret;
}
void syncReconfig(const SSyncNode* pNode, const SSyncCluster* pCfg) {}
int32_t syncAddNode(SSyncNode syncNode, const SNodeInfo *pNode) {
return 0;
}
int32_t syncRemoveNode(SSyncNode syncNode, const SNodeInfo *pNode) {
return 0;
}
// process rpc rsp message from other sync server
static void syncProcessRsp(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) {
}
// process rpc message from other sync server
static void syncProcessReqMsg(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) {
}
static int syncInitRpcServer(SSyncManager* syncManager, const SSyncCluster* pSyncCfg) {
if (gSyncManager->rpcServerTable == NULL) {
gSyncManager->rpcServerTable = taosHashInit(TSDB_MIN_VNODES, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK);
if (gSyncManager->rpcServerTable == NULL) {
syncError("init sync rpc server hash table error");
return -1;
}
}
assert(pSyncCfg->selfIndex < pSyncCfg->replica && pSyncCfg->selfIndex >= 0);
const SNodeInfo* pNode = &(pSyncCfg->nodeInfo[pSyncCfg->replica]);
char buffer[156] = {'\0'};
snprintf(buffer, sizeof(buffer), "%s:%d", &(pNode->nodeFqdn[0]), pNode->nodePort);
size_t len = strlen(buffer);
void** ppRpcServer = taosHashGet(gSyncManager->rpcServerTable, buffer, len);
if (ppRpcServer != NULL) {
// already inited
syncInfo("sync rpc server for %s already exist", buffer);
return 0;
}
SRpcInit rpcInit;
memset(&rpcInit, 0, sizeof(rpcInit));
rpcInit.localPort = pNode->nodePort;
rpcInit.label = "sync-server";
rpcInit.numOfThreads = SYNC_SERVER_WORKER;
rpcInit.cfp = syncProcessReqMsg;
rpcInit.sessions = TSDB_MAX_VNODES << 4;
rpcInit.connType = TAOS_CONN_SERVER;
rpcInit.idleTime = SYNC_ACTIVITY_TIMER * 1000;
void* rpcServer = rpcOpen(&rpcInit);
if (rpcServer == NULL) {
syncInfo("rpcOpen for sync rpc server for %s fail", buffer);
return -1;
}
taosHashPut(gSyncManager->rpcServerTable, buffer, strlen(buffer), rpcServer, len);
syncInfo("sync rpc server for %s init success", buffer);
return 0;
}
static int syncInitRpcClient(SSyncManager* syncManager) {
char secret[TSDB_PASSWORD_LEN] = "secret";
SRpcInit rpcInit;
memset(&rpcInit, 0, sizeof(rpcInit));
rpcInit.label = "sync-client";
rpcInit.numOfThreads = 1;
rpcInit.cfp = syncProcessRsp;
rpcInit.sessions = TSDB_MAX_VNODES << 4;
rpcInit.connType = TAOS_CONN_CLIENT;
rpcInit.idleTime = SYNC_ACTIVITY_TIMER * 1000;
rpcInit.user = "t";
rpcInit.ckey = "key";
rpcInit.secret = secret;
syncManager->clientRpc = rpcOpen(&rpcInit);
if (syncManager->clientRpc == NULL) {
syncError("failed to init sync rpc client");
return -1;
}
syncInfo("sync inter-sync rpc client is initialized");
return 0;
}
static int syncOpenWorkerPool(SSyncManager* syncManager) {
int i;
pthread_attr_t thattr;
pthread_attr_init(&thattr);
pthread_attr_setdetachstate(&thattr, PTHREAD_CREATE_JOINABLE);
for (i = 0; i < TAOS_SYNC_MAX_WORKER; ++i) {
SSyncWorker* pWorker = &(syncManager->worker[i]);
if (pthread_create(&(pWorker->thread), &thattr, (void *)syncWorkerMain, pWorker) != 0) {
syncError("failed to create sync worker since %s", strerror(errno));
return -1;
}
}
pthread_attr_destroy(&thattr);
return 0;
}
static int syncCloseWorkerPool(SSyncManager* syncManager) {
return 0;
}
static void *syncWorkerMain(void *argv) {
SSyncWorker* pWorker = (SSyncWorker *)argv;
taosBlockSIGPIPE();
setThreadName("syncWorker");
return NULL;
}
static void syncNodeTick(void *param, void *tmrId) {
SyncGroupId vgId = (SyncGroupId)((int64_t)param);
SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &vgId, sizeof(SyncGroupId*));
if (ppNode == NULL) {
return;
}
SSyncNode *pNode = *ppNode;
pthread_mutex_lock(&pNode->mutex);
syncRaftTick(&pNode->raft);
pthread_mutex_unlock(&pNode->mutex);
pNode->syncTimer = taosTmrStart(syncNodeTick, SYNC_TICK_TIMER, (void*)(int64_t)pNode->vgId, gSyncManager->syncTimerManager);
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "syncInt.h"
#include "sync_raft_config_change.h"
#include "sync_raft_progress.h"
#include "sync_raft_progress_tracker.h"
#include "sync_raft_quorum_joint.h"
static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
static int checkAndReturn(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap);
static bool hasJointConfig(const SSyncRaftProgressTrackerConfig* config);
static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, const SSyncConfChangeSingleArray* css);
static int symDiff(const SSyncRaftNodeMap* l, const SSyncRaftNodeMap* r);
static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id, bool isLearner);
static void nilAwareDelete(SSyncRaftNodeMap* nodeMap, SyncNodeId id);
static void nilAwareAdd(SSyncRaftNodeMap* nodeMap, SyncNodeId id);
static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id);
static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id);
static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id);
// EnterJoint verifies that the outgoing (=right) majority config of the joint
// config is empty and initializes it with a copy of the incoming (=left)
// majority config. That is, it transitions from
//
// (1 2 3)&&()
// to
// (1 2 3)&&(1 2 3).
//
// The supplied changes are then applied to the incoming majority config,
// resulting in a joint configuration that in terms of the Raft thesis[1]
// (Section 4.3) corresponds to `C_{new,old}`.
//
// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const SSyncConfChangeSingleArray* css,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
int ret;
ret = checkAndCopy(changer, config, progressMap);
if (ret != 0) {
return ret;
}
if (hasJointConfig(config)) {
syncError("config is already joint");
return -1;
}
if(syncRaftJointConfigIsIncomingEmpty(&config->voters) == 0) {
// We allow adding nodes to an empty config for convenience (testing and
// bootstrap), but you can't enter a joint state.
syncError("can't make a zero-voter config joint");
return -1;
}
// Clear the outgoing config.
syncRaftJointConfigClearOutgoing(&config->voters);
// Copy incoming to outgoing.
syncRaftCopyNodeMap(&config->voters.incoming, &config->voters.outgoing);
ret = applyConfig(changer, config, progressMap, css);
if (ret != 0) {
return ret;
}
config->autoLeave = autoLeave;
return checkAndReturn(config, progressMap);
}
// Simple carries out a series of configuration changes that (in aggregate)
// mutates the incoming majority config Voters[0] by at most one. This method
// will return an error if that is not the case, if the resulting quorum is
// zero, or if the configuration is in a joint state (i.e. if there is an
// outgoing configuration).
int syncRaftChangerSimpleConfig(SSyncRaftChanger* changer, const SSyncConfChangeSingleArray* css,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
int ret;
ret = checkAndCopy(changer, config, progressMap);
if (ret != 0) {
return ret;
}
if (hasJointConfig(config)) {
syncError("can't apply simple config change in joint config");
return -1;
}
ret = applyConfig(changer, config, progressMap, css);
if (ret != 0) {
return ret;
}
int n = symDiff(syncRaftJointConfigIncoming(&changer->tracker->config.voters),
syncRaftJointConfigIncoming(&config->voters));
if (n > 1) {
syncError("more than one voter changed without entering joint config");
return -1;
}
return checkAndReturn(config, progressMap);
}
// apply a change to the configuration. By convention, changes to voters are
// always made to the incoming majority config Voters[0]. Voters[1] is either
// empty or preserves the outgoing majority configuration while in a joint state.
static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, const SSyncConfChangeSingleArray* css) {
int i;
for (i = 0; i < css->n; ++i) {
const SSyncConfChangeSingle* cs = &(css->changes[i]);
if (cs->nodeId == SYNC_NON_NODE_ID) {
continue;
}
ESyncRaftConfChangeType type = cs->type;
switch (type) {
case SYNC_RAFT_Conf_AddNode:
makeVoter(changer, config, progressMap, cs->nodeId);
break;
case SYNC_RAFT_Conf_AddLearnerNode:
makeLearner(changer, config, progressMap, cs->nodeId);
break;
case SYNC_RAFT_Conf_RemoveNode:
removeNodeId(changer, config, progressMap, cs->nodeId);
break;
case SYNC_RAFT_Conf_UpdateNode:
break;
}
}
if (syncRaftJointConfigIsIncomingEmpty(&config->voters)) {
syncError("removed all voters");
return -1;
}
return 0;
}
// makeVoter adds or promotes the given ID to be a voter in the incoming
// majority config.
static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id) {
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id);
if (progress == NULL) {
initProgress(changer, config, progressMap, id, false);
return;
}
progress->isLearner = false;
nilAwareDelete(&config->learners, id);
nilAwareDelete(&config->learnersNext, id);
syncRaftJointConfigAddToIncoming(&config->voters, id);
}
// makeLearner makes the given ID a learner or stages it to be a learner once
// an active joint configuration is exited.
//
// The former happens when the peer is not a part of the outgoing config, in
// which case we either add a new learner or demote a voter in the incoming
// config.
//
// The latter case occurs when the configuration is joint and the peer is a
// voter in the outgoing config. In that case, we do not want to add the peer
// as a learner because then we'd have to track a peer as a voter and learner
// simultaneously. Instead, we add the learner to LearnersNext, so that it will
// be added to Learners the moment the outgoing config is removed by
// LeaveJoint().
static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id) {
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id);
if (progress == NULL) {
initProgress(changer, config, progressMap, id, true);
return;
}
if (progress->isLearner) {
return;
}
// Remove any existing voter in the incoming config...
removeNodeId(changer, config, progressMap, id);
// ... but save the Progress.
syncRaftAddToProgressMap(progressMap, progress);
// Use LearnersNext if we can't add the learner to Learners directly, i.e.
// if the peer is still tracked as a voter in the outgoing config. It will
// be turned into a learner in LeaveJoint().
//
// Otherwise, add a regular learner right away.
bool inInOutgoing = syncRaftJointConfigIsInOutgoing(&config->voters, id);
if (inInOutgoing) {
nilAwareAdd(&config->learnersNext, id);
} else {
nilAwareAdd(&config->learners, id);
progress->isLearner = true;
}
}
// removeNodeId this peer as a voter or learner from the incoming config.
static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id) {
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id);
if (progress == NULL) {
return;
}
syncRaftJointConfigRemoveFromIncoming(&config->voters, id);
nilAwareDelete(&config->learners, id);
nilAwareDelete(&config->learnersNext, id);
// If the peer is still a voter in the outgoing config, keep the Progress.
bool inInOutgoing = syncRaftJointConfigIsInOutgoing(&config->voters, id);
if (!inInOutgoing) {
syncRaftRemoveFromProgressMap(progressMap, id);
}
}
// initProgress initializes a new progress for the given node or learner.
static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config,
SSyncRaftProgressMap* progressMap, SyncNodeId id, bool isLearner) {
if (!isLearner) {
syncRaftJointConfigAddToIncoming(&config->voters, id);
} else {
nilAwareAdd(&config->learners, id);
}
SSyncRaftProgress* pProgress = (SSyncRaftProgress*)malloc(sizeof(SSyncRaftProgress));
assert (pProgress != NULL);
*pProgress = (SSyncRaftProgress) {
// Initializing the Progress with the last index means that the follower
// can be probed (with the last index).
//
// TODO(tbg): seems awfully optimistic. Using the first index would be
// better. The general expectation here is that the follower has no log
// at all (and will thus likely need a snapshot), though the app may
// have applied a snapshot out of band before adding the replica (thus
// making the first index the better choice).
.id = id,
.groupId = changer->tracker->pRaft->selfGroupId,
.nextIndex = changer->lastIndex,
.matchIndex = 0,
.state = PROGRESS_STATE_PROBE,
.pendingSnapshotIndex = 0,
.probeSent = false,
.inflights = syncRaftOpenInflights(changer->tracker->maxInflightMsgs),
.isLearner = isLearner,
// When a node is first added, we should mark it as recently active.
// Otherwise, CheckQuorum may cause us to step down if it is invoked
// before the added node has had a chance to communicate with us.
.recentActive = true,
.refCount = 0,
};
syncRaftAddToProgressMap(progressMap, pProgress);
}
// checkInvariants makes sure that the config and progress are compatible with
// each other. This is used to check both what the Changer is initialized with,
// as well as what it returns.
static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
int ret = syncRaftCheckTrackerConfigInProgress(config, progressMap);
if (ret != 0) {
return ret;
}
// Any staged learner was staged because it could not be directly added due
// to a conflicting voter in the outgoing config.
SyncNodeId* pNodeId = NULL;
while (!syncRaftIterateNodeMap(&config->learnersNext, pNodeId)) {
SyncNodeId nodeId = *pNodeId;
if (!syncRaftJointConfigInOutgoing(&config->voters, nodeId)) {
syncError("[%d] is in LearnersNext, but not outgoing", nodeId);
return -1;
}
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId);
assert(progress);
assert(progress->id == nodeId);
if (progress->isLearner) {
syncError("[%d:%d] is in LearnersNext, but is already marked as learner", progress->groupId, nodeId);
return -1;
}
}
// Conversely Learners and Voters doesn't intersect at all.
pNodeId = NULL;
while (!syncRaftIterateNodeMap(&config->learners, pNodeId)) {
SyncNodeId nodeId = *pNodeId;
if (syncRaftJointConfigInOutgoing(&config->voters, nodeId)) {
syncError("%d is in Learners and outgoing", nodeId);
return -1;
}
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId);
assert(progress);
assert(progress->id == nodeId);
if (!progress->isLearner) {
syncError("[%d:%d] is in Learners, but is not marked as learner", progress->groupId, nodeId);
return -1;
}
}
if (!hasJointConfig(config)) {
// We enforce that empty maps are nil instead of zero.
if (syncRaftNodeMapSize(&config->learnersNext) > 0) {
syncError("cfg.LearnersNext must be nil when not joint");
return -1;
}
if (config->autoLeave) {
syncError("AutoLeave must be false when not joint");
return -1;
}
}
return 0;
}
// checkAndCopy copies the tracker's config and progress map (deeply enough for
// the purposes of the Changer) and returns those copies. It returns an error
// if checkInvariants does.
static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
syncRaftCopyTrackerConfig(&changer->tracker->config, config);
syncRaftClearProgressMap(progressMap);
SSyncRaftProgress* pProgress = NULL;
while (!syncRaftIterateProgressMap(&changer->tracker->progressMap, pProgress)) {
syncRaftAddToProgressMap(progressMap, pProgress);
}
return checkAndReturn(config, progressMap);
}
// checkAndReturn calls checkInvariants on the input and returns either the
// resulting error or the input.
static int checkAndReturn(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
if (checkInvariants(config, progressMap) != 0) {
return -1;
}
return 0;
}
static bool hasJointConfig(const SSyncRaftProgressTrackerConfig* config) {
return !syncRaftJointConfigIsOutgoingEmpty(&config->voters);
}
// symdiff returns the count of the symmetric difference between the sets of
// uint64s, i.e. len( (l - r) \union (r - l)).
static int symDiff(const SSyncRaftNodeMap* l, const SSyncRaftNodeMap* r) {
int n;
int i;
int j0, j1;
const SSyncRaftNodeMap* pairs[2][2] = {
{l, r}, // count elems in l but not in r
{r, l}, // count elems in r but not in l
};
for (n = 0, i = 0; i < 2; ++i) {
const SSyncRaftNodeMap** pp = pairs[i];
const SSyncRaftNodeMap* p0 = pp[0];
const SSyncRaftNodeMap* p1 = pp[1];
SyncNodeId* pNodeId;
while (!syncRaftIterateNodeMap(p0, pNodeId)) {
if (!syncRaftIsInNodeMap(p1, *pNodeId)) {
n+=1;
}
}
}
return n;
}
// nilAwareDelete deletes from a map, nil'ing the map itself if it is empty after.
static void nilAwareDelete(SSyncRaftNodeMap* nodeMap, SyncNodeId id) {
syncRaftRemoveFromNodeMap(nodeMap, id);
}
// nilAwareAdd populates a map entry, creating the map if necessary.
static void nilAwareAdd(SSyncRaftNodeMap* nodeMap, SyncNodeId id) {
syncRaftAddToNodeMap(nodeMap, id);
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "syncInt.h"
#include "raft.h"
#include "raft_log.h"
#include "raft_message.h"
#include "sync_raft_progress_tracker.h"
void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) {
if (pRaft->state == TAOS_SYNC_STATE_LEADER) {
syncDebug("[%d:%d] ignoring RAFT_MSG_INTERNAL_ELECTION because already leader", pRaft->selfGroupId, pRaft->selfId);
return;
}
if (!syncRaftIsPromotable(pRaft)) {
syncWarn("[%d:%d] is unpromotable and can not syncRaftCampaign", pRaft->selfGroupId, pRaft->selfId);
return;
}
// if there is pending uncommitted config,cannot start election
if (syncRaftLogNumOfPendingConf(pRaft->log) > 0 && syncRaftHasUnappliedLog(pRaft->log)) {
syncWarn("[%d:%d] cannot syncRaftStartElection at term %" PRId64 " since there are still pending configuration changes to apply",
pRaft->selfGroupId, pRaft->selfId, pRaft->term);
return;
}
syncInfo("[%d:%d] is starting a new election at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term);
syncRaftCampaign(pRaft, cType);
}
// syncRaftCampaign transitions the raft instance to candidate state. This must only be
// called after verifying that this is a legitimate transition.
void syncRaftCampaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) {
bool preVote;
SyncTerm term;
if (syncRaftIsPromotable(pRaft)) {
syncDebug("[%d:%d] is unpromotable; syncRaftCampaign() should have been called", pRaft->selfGroupId, pRaft->selfId);
return;
}
if (cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION) {
syncRaftBecomePreCandidate(pRaft);
preVote = true;
// PreVote RPCs are sent for the next term before we've incremented r.Term.
term = pRaft->term + 1;
} else {
syncRaftBecomeCandidate(pRaft);
term = pRaft->term;
preVote = false;
}
int quorum = syncRaftQuorum(pRaft);
ESyncRaftVoteResult result = syncRaftPollVote(pRaft, pRaft->selfId, preVote, true, NULL, NULL);
if (result == SYNC_RAFT_VOTE_WON) {
// We won the election after voting for ourselves (which must mean that
// this is a single-node cluster). Advance to the next state.
if (cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION) {
syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION);
} else {
syncRaftBecomeLeader(pRaft);
}
return;
}
// broadcast vote message to other peers
int i;
SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log);
SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log);
SSyncRaftNodeMap nodeMap;
syncRaftJointConfigIDs(&pRaft->tracker->config.voters, &nodeMap);
SyncNodeId *pNodeId = NULL;
while (!syncRaftIterateNodeMap(&nodeMap, pNodeId)) {
SyncNodeId nodeId = *pNodeId;
if (nodeId == SYNC_NON_NODE_ID) {
continue;
}
if (nodeId == pRaft->selfId) {
continue;
}
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, nodeId);
if (pNode == NULL) {
continue;
}
SSyncMessage* pMsg = syncNewVoteMsg(pRaft->selfGroupId, pRaft->selfId,
term, cType, lastIndex, lastTerm);
if (pMsg == NULL) {
continue;
}
syncInfo("[%d:%d] [logterm: %" PRId64 ", index: %" PRId64 "] sent vote request to %d at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, lastTerm,
lastIndex, nodeId, pRaft->term);
pRaft->io.send(pMsg, pNode);
}
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "sync_raft_impl.h"
#include "raft_log.h"
#include "raft_replication.h"
#include "sync_raft_progress_tracker.h"
#include "syncInt.h"
static int convertClear(SSyncRaft* pRaft);
static int stepFollower(SSyncRaft* pRaft, const SSyncMessage* pMsg);
static int stepCandidate(SSyncRaft* pRaft, const SSyncMessage* pMsg);
static int stepLeader(SSyncRaft* pRaft, const SSyncMessage* pMsg);
static bool increaseUncommittedSize(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n);
static int triggerAll(SSyncRaft* pRaft);
static void tickElection(SSyncRaft* pRaft);
static void tickHeartbeat(SSyncRaft* pRaft);
static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n);
static void abortLeaderTransfer(SSyncRaft* pRaft);
static void resetRaft(SSyncRaft* pRaft, SyncTerm term);
void syncRaftBecomeFollower(SSyncRaft* pRaft, SyncTerm term, SyncNodeId leaderId) {
convertClear(pRaft);
pRaft->stepFp = stepFollower;
resetRaft(pRaft, term);
pRaft->tickFp = tickElection;
pRaft->leaderId = leaderId;
pRaft->state = TAOS_SYNC_STATE_FOLLOWER;
syncInfo("[%d:%d] became followe at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term);
}
void syncRaftBecomePreCandidate(SSyncRaft* pRaft) {
convertClear(pRaft);
/**
* Becoming a pre-candidate changes our step functions and state,
* but doesn't change anything else. In particular it does not increase
* r.Term or change r.Vote.
**/
pRaft->stepFp = stepCandidate;
pRaft->tickFp = tickElection;
pRaft->state = TAOS_SYNC_STATE_CANDIDATE;
pRaft->candidateState.inPreVote = true;
syncInfo("[%d:%d] became pre-candidate at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term);
}
void syncRaftBecomeCandidate(SSyncRaft* pRaft) {
convertClear(pRaft);
pRaft->candidateState.inPreVote = false;
pRaft->stepFp = stepCandidate;
// become candidate make term+1
resetRaft(pRaft, pRaft->term + 1);
pRaft->tickFp = tickElection;
pRaft->voteFor = pRaft->selfId;
pRaft->state = TAOS_SYNC_STATE_CANDIDATE;
syncInfo("[%d:%d] became candidate at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term);
}
void syncRaftBecomeLeader(SSyncRaft* pRaft) {
assert(pRaft->state != TAOS_SYNC_STATE_FOLLOWER);
pRaft->stepFp = stepLeader;
resetRaft(pRaft, pRaft->term);
pRaft->leaderId = pRaft->leaderId;
pRaft->state = TAOS_SYNC_STATE_LEADER;
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, pRaft->selfId);
assert(progress != NULL);
// Followers enter replicate mode when they've been successfully probed
// (perhaps after having received a snapshot as a result). The leader is
// trivially in this state. Note that r.reset() has initialized this
// progress with the last index already.
syncRaftProgressBecomeReplicate(progress);
// Conservatively set the pendingConfIndex to the last index in the
// log. There may or may not be a pending config change, but it's
// safe to delay any future proposals until we commit all our
// pending log entries, and scanning the entire tail of the log
// could be expensive.
SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log);
pRaft->pendingConfigIndex = lastIndex;
// after become leader, send a no-op log
SSyncRaftEntry* entry = (SSyncRaftEntry*)malloc(sizeof(SSyncRaftEntry));
if (entry == NULL) {
return;
}
*entry = (SSyncRaftEntry) {
.buffer = (SSyncBuffer) {
.data = NULL,
.len = 0,
}
};
appendEntries(pRaft, entry, 1);
//syncRaftTriggerHeartbeat(pRaft);
syncInfo("[%d:%d] became leader at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term);
}
void syncRaftTriggerHeartbeat(SSyncRaft* pRaft) {
triggerAll(pRaft);
}
void syncRaftRandomizedElectionTimeout(SSyncRaft* pRaft) {
// electionTimeoutTick in [3,6] tick
pRaft->randomizedElectionTimeout = taosRand() % 4 + 3;
}
bool syncRaftIsPromotable(SSyncRaft* pRaft) {
return pRaft->selfId != SYNC_NON_NODE_ID;
}
bool syncRaftIsPastElectionTimeout(SSyncRaft* pRaft) {
return pRaft->electionElapsed >= pRaft->randomizedElectionTimeout;
}
int syncRaftQuorum(SSyncRaft* pRaft) {
return 0;
//return pRaft->cluster.replica / 2 + 1;
}
ESyncRaftVoteResult syncRaftPollVote(SSyncRaft* pRaft, SyncNodeId id,
bool preVote, bool grant,
int* rejected, int *granted) {
SNodeInfo* pNode = syncRaftGetNodeById(pRaft, id);
if (pNode == NULL) {
return true;
}
if (grant) {
syncInfo("[%d:%d] received grant (pre-vote %d) from %d at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, preVote, id, pRaft->term);
} else {
syncInfo("[%d:%d] received rejection (pre-vote %d) from %d at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, preVote, id, pRaft->term);
}
syncRaftRecordVote(pRaft->tracker, pNode->nodeId, grant);
return syncRaftTallyVotes(pRaft->tracker, rejected, granted);
}
/*
if (accept) {
syncInfo("[%d:%d] received (pre-vote %d) from %d at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, preVote, id, pRaft->term);
} else {
syncInfo("[%d:%d] received rejection from %d at term %" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, id, pRaft->term);
}
int voteIndex = syncRaftGetNodeById(pRaft, id);
assert(voteIndex < pRaft->cluster.replica && voteIndex >= 0);
assert(pRaft->candidateState.votes[voteIndex] == SYNC_RAFT_VOTE_RESP_UNKNOWN);
pRaft->candidateState.votes[voteIndex] = accept ? SYNC_RAFT_VOTE_RESP_GRANT : SYNC_RAFT_VOTE_RESP_REJECT;
int granted = 0, rejected = 0;
int i;
for (i = 0; i < pRaft->cluster.replica; ++i) {
if (pRaft->candidateState.votes[i] == SYNC_RAFT_VOTE_RESP_GRANT) granted++;
else if (pRaft->candidateState.votes[i] == SYNC_RAFT_VOTE_RESP_REJECT) rejected++;
}
if (rejectNum) *rejectNum = rejected;
return granted;
*/
void syncRaftLoadState(SSyncRaft* pRaft, const SSyncServerState* serverState) {
SyncIndex commitIndex = serverState->commitIndex;
SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log);
if (commitIndex < pRaft->log->commitIndex || commitIndex > lastIndex) {
syncFatal("[%d:%d] state.commit %"PRId64" is out of range [%" PRId64 ",%" PRId64 "",
pRaft->selfGroupId, pRaft->selfId, commitIndex, pRaft->log->commitIndex, lastIndex);
return;
}
pRaft->log->commitIndex = commitIndex;
pRaft->term = serverState->term;
pRaft->voteFor = serverState->voteFor;
}
static void visitProgressSendAppend(SSyncRaftProgress* progress, void* arg) {
SSyncRaft* pRaft = (SSyncRaft*)arg;
if (pRaft->selfId == progress->id) {
return;
}
syncRaftMaybeSendAppend(arg, progress, true);
}
// bcastAppend sends RPC, with entries to all peers that are not up-to-date
// according to the progress recorded in r.prs.
void syncRaftBroadcastAppend(SSyncRaft* pRaft) {
syncRaftProgressVisit(pRaft->tracker, visitProgressSendAppend, pRaft);
}
SNodeInfo* syncRaftGetNodeById(SSyncRaft *pRaft, SyncNodeId id) {
SNodeInfo **ppNode = taosHashGet(pRaft->nodeInfoMap, &id, sizeof(SyncNodeId*));
if (ppNode != NULL) {
return *ppNode;
}
return NULL;
}
static int convertClear(SSyncRaft* pRaft) {
}
static int stepFollower(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
return 0;
}
static int stepCandidate(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
/**
* Only handle vote responses corresponding to our candidacy (while in
* StateCandidate, we may get stale MsgPreVoteResp messages in this term from
* our pre-candidate state).
**/
ESyncRaftMessageType msgType = pMsg->msgType;
if (msgType == RAFT_MSG_INTERNAL_PROP) {
return 0;
}
if (msgType == RAFT_MSG_VOTE_RESP) {
syncRaftHandleVoteRespMessage(pRaft, pMsg);
return 0;
} else if (msgType == RAFT_MSG_APPEND) {
syncRaftBecomeFollower(pRaft, pMsg->term, pMsg->from);
syncRaftHandleAppendEntriesMessage(pRaft, pMsg);
}
return 0;
}
static int stepLeader(SSyncRaft* pRaft, const SSyncMessage* pMsg) {
convertClear(pRaft);
return 0;
}
// tickElection is run by followers and candidates after r.electionTimeout.
static void tickElection(SSyncRaft* pRaft) {
pRaft->electionElapsed += 1;
if (!syncRaftIsPromotable(pRaft)) {
return;
}
if (!syncRaftIsPastElectionTimeout(pRaft)) {
return;
}
// election timeout
pRaft->electionElapsed = 0;
SSyncMessage msg;
syncRaftStep(pRaft, syncInitElectionMsg(&msg, pRaft->selfId));
}
// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
static void tickHeartbeat(SSyncRaft* pRaft) {
}
// TODO
static bool increaseUncommittedSize(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) {
return false;
}
static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) {
SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log);
SyncTerm term = pRaft->term;
int i;
for (i = 0; i < n; ++i) {
entries[i].term = term;
entries[i].index = lastIndex + 1 + i;
}
// Track the size of this uncommitted proposal.
if (!increaseUncommittedSize(pRaft, entries, n)) {
// Drop the proposal.
return;
}
syncRaftLogAppend(pRaft->log, entries, n);
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, pRaft->selfId);
assert(progress != NULL);
syncRaftProgressMaybeUpdate(progress, lastIndex);
// Regardless of syncRaftMaybeCommit's return, our caller will call bcastAppend.
syncRaftMaybeCommit(pRaft);
}
// syncRaftMaybeCommit attempts to advance the commit index. Returns true if
// the commit index changed (in which case the caller should call
// r.bcastAppend).
bool syncRaftMaybeCommit(SSyncRaft* pRaft) {
return true;
}
/**
* trigger I/O requests for newly appended log entries or heartbeats.
**/
static int triggerAll(SSyncRaft* pRaft) {
#if 0
assert(pRaft->state == TAOS_SYNC_STATE_LEADER);
int i;
for (i = 0; i < pRaft->cluster.replica; ++i) {
if (i == pRaft->cluster.selfIndex) {
continue;
}
syncRaftMaybeSendAppend(pRaft, pRaft->tracker->progressMap.progress[i], true);
}
#endif
return 0;
}
static void abortLeaderTransfer(SSyncRaft* pRaft) {
pRaft->leadTransferee = SYNC_NON_NODE_ID;
}
static void resetProgress(SSyncRaftProgress* progress, void* arg) {
syncRaftResetProgress((SSyncRaft*)arg, progress);
}
static void resetRaft(SSyncRaft* pRaft, SyncTerm term) {
if (pRaft->term != term) {
pRaft->term = term;
pRaft->voteFor = SYNC_NON_NODE_ID;
}
pRaft->leaderId = SYNC_NON_NODE_ID;
pRaft->electionElapsed = 0;
pRaft->heartbeatElapsed = 0;
syncRaftRandomizedElectionTimeout(pRaft);
abortLeaderTransfer(pRaft);
syncRaftResetVotes(pRaft->tracker);
syncRaftProgressVisit(pRaft->tracker, resetProgress, pRaft);
pRaft->pendingConfigIndex = 0;
pRaft->uncommittedSize = 0;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http: *www.gnu.org/licenses/>.
*/
#include "sync_raft_inflights.h"
SSyncRaftInflights* syncRaftOpenInflights(int size) {
SSyncRaftInflights* inflights = (SSyncRaftInflights*)malloc(sizeof(SSyncRaftInflights));
if (inflights == NULL) {
return NULL;
}
SyncIndex* buffer = (SyncIndex*)malloc(sizeof(SyncIndex) * size);
if (buffer == NULL) {
free(inflights);
return NULL;
}
*inflights = (SSyncRaftInflights) {
.buffer = buffer,
.count = 0,
.size = 0,
.start = 0,
};
return inflights;
}
void syncRaftCloseInflights(SSyncRaftInflights* inflights) {
free(inflights->buffer);
free(inflights);
}
// Add notifies the Inflights that a new message with the given index is being
// dispatched. Full() must be called prior to Add() to verify that there is room
// for one more message, and consecutive calls to add Add() must provide a
// monotonic sequence of indexes.
void syncRaftInflightAdd(SSyncRaftInflights* inflights, SyncIndex inflightIndex) {
assert(!syncRaftInflightFull(inflights));
int next = inflights->start + inflights->count;
int size = inflights->size;
if (next >= size) {
next -= size;
}
inflights->buffer[next] = inflightIndex;
inflights->count++;
}
// FreeLE frees the inflights smaller or equal to the given `to` flight.
void syncRaftInflightFreeLE(SSyncRaftInflights* inflights, SyncIndex toIndex) {
if (inflights->count == 0 || toIndex < inflights->buffer[inflights->start]) {
// out of the left side of the window
return;
}
int i, idx;
for (i = 0, idx = inflights->start; i < inflights->count; i++) {
if (toIndex < inflights->buffer[idx]) { // found the first large inflight
break;
}
// increase index and maybe rotate
int size = inflights->size;
idx++;
if (idx >= size) {
idx -= size;
}
}
// free i inflights and set new start index
inflights->count -= i;
inflights->start = idx;
assert(inflights->count >= 0);
if (inflights->count == 0) {
// inflights is empty, reset the start index so that we don't grow the
// buffer unnecessarily.
inflights->start = 0;
}
}
// FreeFirstOne releases the first inflight. This is a no-op if nothing is
// inflight.
void syncRaftInflightFreeFirstOne(SSyncRaftInflights* inflights) {
syncRaftInflightFreeLE(inflights, inflights->buffer[inflights->start]);
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "sync_raft_node_map.h"
#include "sync_type.h"
#include "sync_raft_progress.h"
void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap) {
nodeMap->nodeIdMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
}
void syncRaftFreeNodeMap(SSyncRaftNodeMap* nodeMap) {
taosHashCleanup(nodeMap->nodeIdMap);
}
void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap) {
taosHashClear(nodeMap->nodeIdMap);
}
bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) {
SyncNodeId** ppId = (SyncNodeId**)taosHashGet(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*));
if (ppId == NULL) {
return false;
}
return true;
}
void syncRaftCopyNodeMap(SSyncRaftNodeMap* from, SSyncRaftNodeMap* to) {
SyncNodeId *pId = NULL;
while (!syncRaftIterateNodeMap(from, pId)) {
taosHashPut(to->nodeIdMap, &pId, sizeof(SyncNodeId*), &pId, sizeof(SyncNodeId*));
}
}
bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId) {
SyncNodeId **ppId = taosHashIterate(nodeMap->nodeIdMap, pId);
if (ppId == NULL) {
return true;
}
*pId = *(*ppId);
return false;
}
bool syncRaftIsAllNodeInProgressMap(SSyncRaftNodeMap* nodeMap, SSyncRaftProgressMap* progressMap) {
SyncNodeId *pId = NULL;
while (!syncRaftIterateNodeMap(nodeMap, pId)) {
if (!syncRaftIsInProgressMap(progressMap, *pId)) {
return false;
}
}
return true;
}
void syncRaftUnionNodeMap(SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) {
syncRaftCopyNodeMap(nodeMap, to);
}
void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) {
taosHashPut(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*), &nodeId, sizeof(SyncNodeId*));
}
void syncRaftRemoveFromNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) {
taosHashRemove(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*));
}
int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap) {
return taosHashGetSize(nodeMap->nodeIdMap);
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "raft_log.h"
#include "sync_raft_progress.h"
#include "sync_raft_progress_tracker.h"
#include "sync.h"
#include "syncInt.h"
static void copyProgress(SSyncRaftProgress* progress, void* arg);
static void refProgress(SSyncRaftProgress* progress);
static void unrefProgress(SSyncRaftProgress* progress, void*);
static void resetProgressState(SSyncRaftProgress* progress, ESyncRaftProgressState state);
static void probeAcked(SSyncRaftProgress* progress);
static void resumeProgress(SSyncRaftProgress* progress);
void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress) {
if (progress->inflights) {
syncRaftCloseInflights(progress->inflights);
}
SSyncRaftInflights* inflights = syncRaftOpenInflights(pRaft->tracker->maxInflightMsgs);
if (inflights == NULL) {
return;
}
*progress = (SSyncRaftProgress) {
.matchIndex = progress->id == pRaft->selfId ? syncRaftLogLastIndex(pRaft->log) : 0,
.nextIndex = syncRaftLogLastIndex(pRaft->log) + 1,
.inflights = inflights,
.isLearner = false,
.state = PROGRESS_STATE_PROBE,
};
}
// MaybeUpdate is called when an MsgAppResp arrives from the follower, with the
// index acked by it. The method returns false if the given n index comes from
// an outdated message. Otherwise it updates the progress and returns true.
bool syncRaftProgressMaybeUpdate(SSyncRaftProgress* progress, SyncIndex lastIndex) {
bool updated = false;
if (progress->matchIndex < lastIndex) {
progress->matchIndex = lastIndex;
updated = true;
probeAcked(progress);
}
progress->nextIndex = TMAX(progress->nextIndex, lastIndex + 1);
return updated;
}
// MaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The
// arguments are the index of the append message rejected by the follower, and
// the hint that we want to decrease to.
//
// Rejections can happen spuriously as messages are sent out of order or
// duplicated. In such cases, the rejection pertains to an index that the
// Progress already knows were previously acknowledged, and false is returned
// without changing the Progress.
//
// If the rejection is genuine, Next is lowered sensibly, and the Progress is
// cleared for sending log entries.
bool syncRaftProgressMaybeDecrTo(SSyncRaftProgress* progress,
SyncIndex rejected, SyncIndex matchHint) {
if (progress->state == PROGRESS_STATE_REPLICATE) {
// The rejection must be stale if the progress has matched and "rejected"
// is smaller than "match".
if (rejected <= progress->matchIndex) {
syncDebug("match index is up to date,ignore");
return false;
}
// Directly decrease next to match + 1.
//
// TODO(tbg): why not use matchHint if it's larger?
progress->nextIndex = progress->matchIndex + 1;
return true;
}
// The rejection must be stale if "rejected" does not match next - 1. This
// is because non-replicating followers are probed one entry at a time.
if (rejected != progress->nextIndex - 1) {
syncDebug("rejected index %" PRId64 " different from next index %" PRId64 " -> ignore"
, rejected, progress->nextIndex);
return false;
}
progress->nextIndex = TMAX(TMIN(rejected, matchHint + 1), 1);
progress->probeSent = false;
return true;
}
// IsPaused returns whether sending log entries to this node has been throttled.
// This is done when a node has rejected recent MsgApps, is currently waiting
// for a snapshot, or has reached the MaxInflightMsgs limit. In normal
// operation, this is false. A throttled node will be contacted less frequently
// until it has reached a state in which it's able to accept a steady stream of
// log entries again.
bool syncRaftProgressIsPaused(SSyncRaftProgress* progress) {
switch (progress->state) {
case PROGRESS_STATE_PROBE:
return progress->probeSent;
case PROGRESS_STATE_REPLICATE:
return syncRaftInflightFull(progress->inflights);
case PROGRESS_STATE_SNAPSHOT:
return true;
default:
syncFatal("error sync state:%d", progress->state);
}
}
SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id) {
SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*));
if (ppProgress == NULL) {
return NULL;
}
return *ppProgress;
}
int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress) {
refProgress(progress);
taosHashPut(progressMap->progressMap, &progress->id, sizeof(SyncNodeId*), &progress, sizeof(SSyncRaftProgress*));
}
void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) {
SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*));
if (ppProgress == NULL) {
return;
}
unrefProgress(*ppProgress, NULL);
taosHashRemove(progressMap->progressMap, &id, sizeof(SyncNodeId*));
}
bool syncRaftIsInProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) {
return taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*)) != NULL;
}
bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress) {
return syncRaftLogLastIndex(pRaft->log) + 1 == progress->nextIndex;
}
// BecomeProbe transitions into StateProbe. Next is reset to Match+1 or,
// optionally and if larger, the index of the pending snapshot.
void syncRaftProgressBecomeProbe(SSyncRaftProgress* progress) {
// If the original state is StateSnapshot, progress knows that
// the pending snapshot has been sent to this peer successfully, then
// probes from pendingSnapshot + 1.
if (progress->state == PROGRESS_STATE_SNAPSHOT) {
SyncIndex pendingSnapshotIndex = progress->pendingSnapshotIndex;
resetProgressState(progress, PROGRESS_STATE_PROBE);
progress->nextIndex = TMAX(progress->matchIndex + 1, pendingSnapshotIndex + 1);
} else {
resetProgressState(progress, PROGRESS_STATE_PROBE);
progress->nextIndex = progress->matchIndex + 1;
}
}
// BecomeReplicate transitions into StateReplicate, resetting Next to Match+1.
void syncRaftProgressBecomeReplicate(SSyncRaftProgress* progress) {
resetProgressState(progress, PROGRESS_STATE_REPLICATE);
progress->nextIndex = progress->matchIndex + 1;
}
// BecomeSnapshot moves the Progress to StateSnapshot with the specified pending
// snapshot index.
void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snapshotIndex) {
resetProgressState(progress, PROGRESS_STATE_SNAPSHOT);
progress->pendingSnapshotIndex = snapshotIndex;
}
void syncRaftCopyProgress(const SSyncRaftProgress* progress, SSyncRaftProgress* out) {
memcpy(out, progress, sizeof(SSyncRaftProgress));
}
void syncRaftInitProgressMap(SSyncRaftProgressMap* progressMap) {
progressMap->progressMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
}
void syncRaftFreeProgressMap(SSyncRaftProgressMap* progressMap) {
syncRaftVisitProgressMap(progressMap, unrefProgress, NULL);
taosHashCleanup(progressMap->progressMap);
}
void syncRaftClearProgressMap(SSyncRaftProgressMap* progressMap) {
taosHashClear(progressMap->progressMap);
}
void syncRaftCopyProgressMap(SSyncRaftProgressMap* from, SSyncRaftProgressMap* to) {
syncRaftVisitProgressMap(from, copyProgress, to);
}
bool syncRaftIterateProgressMap(const SSyncRaftProgressMap* progressMap, SSyncRaftProgress *pProgress) {
SSyncRaftProgress **ppProgress = taosHashIterate(progressMap->progressMap, pProgress);
if (ppProgress == NULL) {
return true;
}
*pProgress = *(*ppProgress);
return false;
}
bool syncRaftVisitProgressMap(SSyncRaftProgressMap* progressMap, visitProgressFp fp, void* arg) {
SSyncRaftProgress *pProgress;
while (!syncRaftIterateProgressMap(progressMap, pProgress)) {
fp(pProgress, arg);
}
}
static void copyProgress(SSyncRaftProgress* progress, void* arg) {
assert(progress->refCount > 0);
SSyncRaftProgressMap* to = (SSyncRaftProgressMap*)arg;
syncRaftAddToProgressMap(to, progress);
}
static void refProgress(SSyncRaftProgress* progress) {
progress->refCount += 1;
}
static void unrefProgress(SSyncRaftProgress* progress, void* arg) {
(void)arg;
progress->refCount -= 1;
assert(progress->refCount >= 0);
if (progress->refCount == 0) {
free(progress);
}
}
// ResetState moves the Progress into the specified State, resetting ProbeSent,
// PendingSnapshot, and Inflights.
static void resetProgressState(SSyncRaftProgress* progress, ESyncRaftProgressState state) {
progress->probeSent = false;
progress->pendingSnapshotIndex = 0;
progress->state = state;
syncRaftInflightReset(progress->inflights);
}
// ProbeAcked is called when this peer has accepted an append. It resets
// ProbeSent to signal that additional append messages should be sent without
// further delay.
static void probeAcked(SSyncRaftProgress* progress) {
progress->probeSent = false;
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "raft.h"
#include "sync_const.h"
#include "sync_raft_progress_tracker.h"
#include "sync_raft_proto.h"
SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft) {
SSyncRaftProgressTracker* tracker = (SSyncRaftProgressTracker*)malloc(sizeof(SSyncRaftProgressTracker));
if (tracker == NULL) {
return NULL;
}
tracker->votesMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
syncRaftInitTrackConfig(&tracker->config);
tracker->pRaft = pRaft;
tracker->maxInflightMsgs = kSyncRaftMaxInflghtMsgs;
return tracker;
}
void syncRaftInitTrackConfig(SSyncRaftProgressTrackerConfig* config) {
syncRaftInitNodeMap(&config->learners);
syncRaftInitNodeMap(&config->learnersNext);
syncRaftInitQuorumJointConfig(&config->voters);
config->autoLeave = false;
}
void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config) {
syncRaftFreeNodeMap(&config->learners);
syncRaftFreeNodeMap(&config->learnersNext);
syncRaftFreeNodeMap(&config->voters.incoming);
syncRaftFreeNodeMap(&config->voters.outgoing);
}
// ResetVotes prepares for a new round of vote counting via recordVote.
void syncRaftResetVotes(SSyncRaftProgressTracker* tracker) {
taosHashClear(tracker->votesMap);
}
void syncRaftProgressVisit(SSyncRaftProgressTracker* tracker, visitProgressFp visit, void* arg) {
syncRaftVisitProgressMap(&tracker->progressMap, visit, arg);
}
// RecordVote records that the node with the given id voted for this Raft
// instance if v == true (and declined it otherwise).
void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant) {
ESyncRaftVoteType* pType = taosHashGet(tracker->votesMap, &id, sizeof(SyncNodeId*));
if (pType != NULL) {
return;
}
taosHashPut(tracker->votesMap, &id, sizeof(SyncNodeId), &grant, sizeof(bool*));
}
void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) {
memcpy(to, from, sizeof(SSyncRaftProgressTrackerConfig));
}
int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
// NB: intentionally allow the empty config. In production we'll never see a
// non-empty config (we prevent it from being created) but we will need to
// be able to *create* an initial config, for example during bootstrap (or
// during tests). Instead of having to hand-code this, we allow
// transitioning from an empty config into any other legal and non-empty
// config.
if (!syncRaftIsAllNodeInProgressMap(&config->voters.incoming, progressMap)) return -1;
if (!syncRaftIsAllNodeInProgressMap(&config->voters.outgoing, progressMap)) return -1;
if (!syncRaftIsAllNodeInProgressMap(&config->learners, progressMap)) return -1;
if (!syncRaftIsAllNodeInProgressMap(&config->learnersNext, progressMap)) return -1;
return 0;
}
// TallyVotes returns the number of granted and rejected Votes, and whether the
// election outcome is known.
ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* rejected, int *granted) {
SSyncRaftProgress* progress = NULL;
int r, g;
// Make sure to populate granted/rejected correctly even if the Votes slice
// contains members no longer part of the configuration. This doesn't really
// matter in the way the numbers are used (they're informational), but might
// as well get it right.
while (!syncRaftIterateProgressMap(&tracker->progressMap, progress)) {
if (progress->id == SYNC_NON_NODE_ID) {
continue;
}
bool* v = taosHashGet(tracker->votesMap, &progress->id, sizeof(SyncNodeId*));
if (v == NULL) {
continue;
}
if (*v) {
g++;
} else {
r++;
}
}
if (rejected) *rejected = r;
if (granted) *granted = g;
return syncRaftVoteResult(&(tracker->config.voters), tracker->votesMap);
}
void syncRaftConfigState(SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) {
syncRaftCopyNodeMap(&tracker->config.voters.incoming, &cs->voters);
syncRaftCopyNodeMap(&tracker->config.voters.outgoing, &cs->votersOutgoing);
syncRaftCopyNodeMap(&tracker->config.learners, &cs->learners);
syncRaftCopyNodeMap(&tracker->config.learnersNext, &cs->learnersNext);
cs->autoLeave = tracker->config.autoLeave;
}
static void matchAckIndexer(SyncNodeId id, void* arg, SyncIndex* index) {
SSyncRaftProgressTracker* tracker = (SSyncRaftProgressTracker*)arg;
SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&tracker->progressMap, id);
if (progress == NULL) {
*index = 0;
return;
}
*index = progress->matchIndex;
}
// Committed returns the largest log index known to be committed based on what
// the voting members of the group have acknowledged.
SyncIndex syncRaftCommittedIndex(SSyncRaftProgressTracker* tracker) {
return syncRaftJointConfigCommittedIndex(&tracker->config.voters, matchAckIndexer, tracker);
}
static void visitProgressActive(SSyncRaftProgress* progress, void* arg) {
SHashObj* votesMap = (SHashObj*)arg;
taosHashPut(votesMap, &progress->id, sizeof(SyncNodeId), &progress->recentActive, sizeof(bool));
}
// QuorumActive returns true if the quorum is active from the view of the local
// raft state machine. Otherwise, it returns false.
bool syncRaftQuorumActive(SSyncRaftProgressTracker* tracker) {
SHashObj* votesMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK);
syncRaftVisitProgressMap(&tracker->progressMap, visitProgressActive, votesMap);
return syncRaftVoteResult(&tracker->config.voters, votesMap) == SYNC_RAFT_VOTE_WON;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "sync_raft_node_map.h"
#include "sync_raft_quorum_majority.h"
#include "sync_raft_quorum_joint.h"
#include "sync_raft_quorum.h"
/**
* syncRaftVoteResult takes a mapping of voters to yes/no (true/false) votes and returns
* a result indicating whether the vote is pending, lost, or won. A joint quorum
* requires both majority quorums to vote in favor.
**/
ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap) {
ESyncRaftVoteResult r1 = syncRaftMajorityVoteResult(&(config->incoming), votesMap);
ESyncRaftVoteResult r2 = syncRaftMajorityVoteResult(&(config->outgoing), votesMap);
if (r1 == r2) {
// If they agree, return the agreed state.
return r1;
}
if (r1 == SYNC_RAFT_VOTE_LOST || r2 == SYNC_RAFT_VOTE_LOST) {
// If either config has lost, loss is the only possible outcome.
return SYNC_RAFT_VOTE_LOST;
}
// One side won, the other one is pending, so the whole outcome is.
return SYNC_RAFT_VOTE_PENDING;
}
void syncRaftInitQuorumJointConfig(SSyncRaftQuorumJointConfig* config) {
syncRaftInitNodeMap(&config->incoming);
syncRaftInitNodeMap(&config->outgoing);
}
void syncRaftFreeQuorumJointConfig(SSyncRaftQuorumJointConfig* config) {
syncRaftFreeNodeMap(&config->incoming);
syncRaftFreeNodeMap(&config->outgoing);
}
void syncRaftJointConfigAddToIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id) {
syncRaftAddToNodeMap(&config->incoming, id);
}
void syncRaftJointConfigRemoveFromIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id) {
syncRaftRemoveFromNodeMap(&config->incoming, id);
}
void syncRaftJointConfigIDs(SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap) {
syncRaftCopyNodeMap(&config->incoming, nodeMap);
syncRaftUnionNodeMap(&config->outgoing, nodeMap);
}
SyncIndex syncRaftJointConfigCommittedIndex(const SSyncRaftQuorumJointConfig* config, matchAckIndexerFp indexer, void* arg) {
SyncIndex index0, index1;
index0 = syncRaftMajorityConfigCommittedIndex(&config->incoming, indexer, arg);
index1 = syncRaftMajorityConfigCommittedIndex(&config->outgoing, indexer, arg);
return index0 < index1 ? index0 : index1;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "sync_const.h"
#include "sync_raft_quorum.h"
#include "sync_raft_quorum_majority.h"
#include "sync_raft_node_map.h"
// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns
// a result indicating whether the vote is pending (i.e. neither a quorum of
// yes/no has been reached), won (a quorum of yes has been reached), or lost (a
// quorum of no has been reached).
ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap) {
int n = syncRaftNodeMapSize(config);
if (n == 0) {
// By convention, the elections on an empty config win. This comes in
// handy with joint quorums because it'll make a half-populated joint
// quorum behave like a majority quorum.
return SYNC_RAFT_VOTE_WON;
}
int i, g, r, missing;
i = g = r = missing = 0;
SyncNodeId* pId = NULL;
while (!syncRaftIterateNodeMap(config, pId)) {
const bool* v = (const bool*)taosHashGet(votesMap, pId, sizeof(SyncNodeId*));
if (v == NULL) {
missing += 1;
continue;
}
if (*v) {
g +=1;
} else {
r += 1;
}
}
int quorum = n / 2 + 1;
if (g >= quorum) {
return SYNC_RAFT_VOTE_WON;
}
if (g + missing >= quorum) {
return SYNC_RAFT_VOTE_PENDING;
}
return SYNC_RAFT_VOTE_LOST;
}
int compSyncIndex(const void * elem1, const void * elem2) {
SyncIndex index1 = *((SyncIndex*)elem1);
SyncIndex index2 = *((SyncIndex*)elem1);
if (index1 > index2) return 1;
if (index1 < index2) return -1;
return 0;
}
SyncIndex syncRaftMajorityConfigCommittedIndex(const SSyncRaftNodeMap* config, matchAckIndexerFp indexer, void* arg) {
int n = syncRaftNodeMapSize(config);
if (n == 0) {
// This plays well with joint quorums which, when one half is the zero
// MajorityConfig, should behave like the other half.
return kMaxCommitIndex;
}
// Use an on-stack slice to collect the committed indexes when n <= 7
// (otherwise we alloc). The alternative is to stash a slice on
// MajorityConfig, but this impairs usability (as is, MajorityConfig is just
// a map, and that's nice). The assumption is that running with a
// replication factor of >7 is rare, and in cases in which it happens
// performance is a lesser concern (additionally the performance
// implications of an allocation here are far from drastic).
SyncIndex* srt = NULL;
SyncIndex srk[TSDB_MAX_REPLICA];
if (n > TSDB_MAX_REPLICA) {
srt = (SyncIndex*)malloc(sizeof(SyncIndex) * n);
if (srt == NULL) {
return kMaxCommitIndex;
}
} else {
srt = &srk[0];
}
// Fill the slice with the indexes observed. Any unused slots will be
// left as zero; these correspond to voters that may report in, but
// haven't yet. We fill from the right (since the zeroes will end up on
// the left after sorting below anyway).
SyncNodeId *pId = NULL;
int i = 0;
SyncIndex index;
while (!syncRaftIterateNodeMap(config, pId)) {
indexer(*pId, arg, &index);
srt[i++] = index;
}
// Sort by index. Use a bespoke algorithm (copied from the stdlib's sort
// package) to keep srt on the stack.
qsort(srt, n, sizeof(SyncIndex), compSyncIndex);
// The smallest index into the array for which the value is acked by a
// quorum. In other words, from the end of the slice, move n/2+1 to the
// left (accounting for zero-indexing).
index = srt[n - (n/2 + 1)];
if (srt != &srk[0]) {
free(srt);
}
return index;
}
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <cli@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "sync_raft_config_change.h"
#include "sync_raft_restore.h"
#include "sync_raft_progress_tracker.h"
static void addToConfChangeSingleArray(SSyncConfChangeSingleArray* out, int* i, const SSyncRaftNodeMap* nodeMap, ESyncRaftConfChangeType t);
static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleArray* out, SSyncConfChangeSingleArray* in);
// syncRaftRestoreConfig takes a Changer (which must represent an empty configuration), and
// runs a sequence of changes enacting the configuration described in the
// ConfState.
//
// TODO(tbg) it's silly that this takes a Changer. Unravel this by making sure
// the Changer only needs a ProgressMap (not a whole Tracker) at which point
// this can just take LastIndex and MaxInflight directly instead and cook up
// the results from that alone.
int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs,
SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) {
SSyncConfChangeSingleArray outgoing;
SSyncConfChangeSingleArray incoming;
SSyncConfChangeSingleArray css;
SSyncRaftProgressTracker* tracker = changer->tracker;
int i, ret;
syncRaftInitConfArray(&outgoing);
syncRaftInitConfArray(&incoming);
syncRaftInitTrackConfig(config);
syncRaftInitProgressMap(progressMap);
ret = toConfChangeSingle(cs, &outgoing, &incoming);
if (ret != 0) {
goto out;
}
if (syncRaftConfArrayIsEmpty(&outgoing)) {
// No outgoing config, so just apply the incoming changes one by one.
for (i = 0; i < incoming.n; ++i) {
css = (SSyncConfChangeSingleArray) {
.n = 1,
.changes = &incoming.changes[i],
};
ret = syncRaftChangerSimpleConfig(changer, &css, config, progressMap);
if (ret != 0) {
goto out;
}
syncRaftCopyTrackerConfig(config, &changer->tracker->config);
syncRaftCopyProgressMap(progressMap, &changer->tracker->progressMap);
}
} else {
// The ConfState describes a joint configuration.
//
// First, apply all of the changes of the outgoing config one by one, so
// that it temporarily becomes the incoming active config. For example,
// if the config is (1 2 3)&(2 3 4), this will establish (2 3 4)&().
for (i = 0; i < outgoing.n; ++i) {
css = (SSyncConfChangeSingleArray) {
.n = 1,
.changes = &outgoing.changes[i],
};
ret = syncRaftChangerSimpleConfig(changer, &css, config, progressMap);
if (ret != 0) {
goto out;
}
syncRaftCopyTrackerConfig(config, &changer->tracker->config);
syncRaftCopyProgressMap(progressMap, &changer->tracker->progressMap);
}
ret = syncRaftChangerEnterJoint(changer, cs->autoLeave, &incoming, config, progressMap);
if (ret != 0) {
goto out;
}
}
out:
syncRaftFreeConfArray(&incoming);
syncRaftFreeConfArray(&outgoing);
return ret;
}
static void addToConfChangeSingleArray(SSyncConfChangeSingleArray* out, int* i, const SSyncRaftNodeMap* nodeMap, ESyncRaftConfChangeType t) {
SyncNodeId* pId = NULL;
while (!syncRaftIterateNodeMap(nodeMap, pId)) {
out->changes[*i] = (SSyncConfChangeSingle) {
.type = t,
.nodeId = *pId,
};
*i += 1;
}
}
// toConfChangeSingle translates a conf state into 1) a slice of operations creating
// first the config that will become the outgoing one, and then the incoming one, and
// b) another slice that, when applied to the config resulted from 1), represents the
// ConfState.
static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleArray* out, SSyncConfChangeSingleArray* in) {
int i;
out->n = syncRaftNodeMapSize(&cs->votersOutgoing);
out->changes = (SSyncConfChangeSingle*)malloc(sizeof(SSyncConfChangeSingle) * out->n);
if (out->changes == NULL) {
out->n = 0;
return -1;
}
in->n = syncRaftNodeMapSize(&cs->votersOutgoing) +
syncRaftNodeMapSize(&cs->voters) +
syncRaftNodeMapSize(&cs->learners) +
syncRaftNodeMapSize(&cs->learnersNext);
out->changes = (SSyncConfChangeSingle*)malloc(sizeof(SSyncConfChangeSingle) * in->n);
if (in->changes == NULL) {
in->n = 0;
return -1;
}
// Example to follow along this code:
// voters=(1 2 3) learners=(5) outgoing=(1 2 4 6) learners_next=(4)
//
// This means that before entering the joint config, the configuration
// had voters (1 2 4 6) and perhaps some learners that are already gone.
// The new set of voters is (1 2 3), i.e. (1 2) were kept around, and (4 6)
// are no longer voters; however 4 is poised to become a learner upon leaving
// the joint state.
// We can't tell whether 5 was a learner before entering the joint config,
// but it doesn't matter (we'll pretend that it wasn't).
//
// The code below will construct
// outgoing = add 1; add 2; add 4; add 6
// incoming = remove 1; remove 2; remove 4; remove 6
// add 1; add 2; add 3;
// add-learner 5;
// add-learner 4;
//
// So, when starting with an empty config, after applying 'outgoing' we have
//
// quorum=(1 2 4 6)
//
// From which we enter a joint state via 'incoming'
//
// quorum=(1 2 3)&&(1 2 4 6) learners=(5) learners_next=(4)
//
// as desired.
// If there are outgoing voters, first add them one by one so that the
// (non-joint) config has them all.
i = 0;
addToConfChangeSingleArray(out, &i, &cs->votersOutgoing, SYNC_RAFT_Conf_AddNode);
assert(i == out->n);
// We're done constructing the outgoing slice, now on to the incoming one
// (which will apply on top of the config created by the outgoing slice).
i = 0;
// First, we'll remove all of the outgoing voters.
addToConfChangeSingleArray(in, &i, &cs->votersOutgoing, SYNC_RAFT_Conf_RemoveNode);
// Then we'll add the incoming voters and learners.
addToConfChangeSingleArray(in, &i, &cs->voters, SYNC_RAFT_Conf_AddNode);
addToConfChangeSingleArray(in, &i, &cs->learners, SYNC_RAFT_Conf_AddLearnerNode);
addToConfChangeSingleArray(in, &i, &cs->learnersNext, SYNC_RAFT_Conf_AddLearnerNode);
assert(i == in->n);
return 0;
}
\ No newline at end of file
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册