diff --git a/source/libs/sync/src/syncAppendEntries.c b/source/libs/sync/src/syncAppendEntries.c index 47d0976aca483ba40480fa730759fe3d1206f568..ba3dd66550c08900fa0b525eba78a2f21413db34 100644 --- a/source/libs/sync/src/syncAppendEntries.c +++ b/source/libs/sync/src/syncAppendEntries.c @@ -89,6 +89,280 @@ // /\ UNCHANGED <> // +int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { + int32_t ret = 0; + + // print log + syncAppendEntriesLog2("==syncNodeOnAppendEntriesCb==", pMsg); + + // if already drop replica, do not process + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { + syncNodeEventLog(ths, "recv sync-append-entries, maybe replica already dropped"); + return ret; + } + + // maybe update term + if (pMsg->term > ths->pRaftStore->currentTerm) { + syncNodeUpdateTerm(ths, pMsg->term); + } + ASSERT(pMsg->term <= ths->pRaftStore->currentTerm); + + // reset elect timer + if (pMsg->term == ths->pRaftStore->currentTerm) { + ths->leaderCache = pMsg->srcId; + syncNodeResetElectTimer(ths); + } + ASSERT(pMsg->dataLen >= 0); + + do { + // return to follower state + if (pMsg->term == ths->pRaftStore->currentTerm && ths->state == TAOS_SYNC_STATE_CANDIDATE) { + syncNodeEventLog(ths, "recv sync-append-entries, candidate to follower"); + + syncNodeBecomeFollower(ths, "from candidate by append entries"); + + // ret or reply? + return ret; + } + } while (0); + + SyncTerm localPreLogTerm = 0; + if (pMsg->prevLogIndex >= SYNC_INDEX_BEGIN && pMsg->prevLogIndex <= ths->pLogStore->getLastIndex(ths->pLogStore)) { + SSyncRaftEntry* pEntry = ths->pLogStore->getEntry(ths->pLogStore, pMsg->prevLogIndex); + if (pEntry == NULL) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "getEntry error, index:%ld, since %s", pMsg->prevLogIndex, terrstr()); + syncNodeErrorLog(ths, logBuf); + return -1; + } + + localPreLogTerm = pEntry->term; + syncEntryDestory(pEntry); + } + + bool logOK = + (pMsg->prevLogIndex == SYNC_INDEX_INVALID) || + ((pMsg->prevLogIndex >= SYNC_INDEX_BEGIN) && + (pMsg->prevLogIndex <= ths->pLogStore->getLastIndex(ths->pLogStore)) && (pMsg->prevLogTerm == localPreLogTerm)); + + // reject request + if ((pMsg->term < ths->pRaftStore->currentTerm) || + ((pMsg->term == ths->pRaftStore->currentTerm) && (ths->state == TAOS_SYNC_STATE_FOLLOWER) && !logOK)) { + do { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "recv sync-append-entries, reject, pre-index:%ld, pre-term:%lu, datalen:%d", + pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->dataLen); + syncNodeEventLog(ths, logBuf); + } while (0); + + SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(ths->vgId); + pReply->srcId = ths->myRaftId; + pReply->destId = pMsg->srcId; + pReply->term = ths->pRaftStore->currentTerm; + pReply->success = false; + pReply->matchIndex = SYNC_INDEX_INVALID; + + SRpcMsg rpcMsg; + syncAppendEntriesReply2RpcMsg(pReply, &rpcMsg); + syncNodeSendMsgById(&pReply->destId, ths, &rpcMsg); + syncAppendEntriesReplyDestroy(pReply); + + return ret; + } + + // accept request + if (pMsg->term == ths->pRaftStore->currentTerm && ths->state == TAOS_SYNC_STATE_FOLLOWER && logOK) { + // preIndex = -1, or has preIndex entry in local log + ASSERT(pMsg->prevLogIndex <= ths->pLogStore->getLastIndex(ths->pLogStore)); + + // has extra entries (> preIndex) in local log + bool hasExtraEntries = pMsg->prevLogIndex < ths->pLogStore->getLastIndex(ths->pLogStore); + + // has entries in SyncAppendEntries msg + bool hasAppendEntries = pMsg->dataLen > 0; + + do { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "recv sync-append-entries, accept, pre-index:%ld, pre-term:%lu, datalen:%d", + pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->dataLen); + syncNodeEventLog(ths, logBuf); + } while (0); + + if (hasExtraEntries && hasAppendEntries) { + // not conflict by default + bool conflict = false; + + SyncIndex extraIndex = pMsg->prevLogIndex + 1; + SSyncRaftEntry* pExtraEntry = ths->pLogStore->getEntry(ths->pLogStore, extraIndex); + if (pExtraEntry == NULL) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "getEntry error2, index:%ld, since %s", extraIndex, terrstr()); + syncNodeErrorLog(ths, logBuf); + return -1; + } + + SSyncRaftEntry* pAppendEntry = syncEntryDeserialize(pMsg->data, pMsg->dataLen); + if (pAppendEntry == NULL) { + syncNodeErrorLog(ths, "syncEntryDeserialize pAppendEntry error"); + return -1; + } + + // log not match, conflict + ASSERT(extraIndex == pAppendEntry->index); + if (pExtraEntry->term != pAppendEntry->term) { + conflict = true; + } + + if (conflict) { + // roll back + SyncIndex delBegin = ths->pLogStore->getLastIndex(ths->pLogStore); + SyncIndex delEnd = extraIndex; + + sTrace("syncNodeOnAppendEntriesCb --> conflict:%d, delBegin:%ld, delEnd:%ld", conflict, delBegin, delEnd); + + // notice! reverse roll back! + for (SyncIndex index = delEnd; index >= delBegin; --index) { + if (ths->pFsm->FpRollBackCb != NULL) { + SSyncRaftEntry* pRollBackEntry = ths->pLogStore->getEntry(ths->pLogStore, index); + if (pRollBackEntry == NULL) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "getEntry error3, index:%ld, since %s", index, terrstr()); + syncNodeErrorLog(ths, logBuf); + return -1; + } + + // if (pRollBackEntry->msgType != TDMT_SYNC_NOOP) { + if (syncUtilUserRollback(pRollBackEntry->msgType)) { + SRpcMsg rpcMsg; + syncEntry2OriginalRpc(pRollBackEntry, &rpcMsg); + + SFsmCbMeta cbMeta = {0}; + cbMeta.index = pRollBackEntry->index; + cbMeta.lastConfigIndex = syncNodeGetSnapshotConfigIndex(ths, cbMeta.index); + cbMeta.isWeak = pRollBackEntry->isWeak; + cbMeta.code = 0; + cbMeta.state = ths->state; + cbMeta.seqNum = pRollBackEntry->seqNum; + ths->pFsm->FpRollBackCb(ths->pFsm, &rpcMsg, cbMeta); + rpcFreeCont(rpcMsg.pCont); + } + + syncEntryDestory(pRollBackEntry); + } + } + + // delete confict entries + ths->pLogStore->truncate(ths->pLogStore, extraIndex); + + // append new entries + ths->pLogStore->appendEntry(ths->pLogStore, pAppendEntry); + + // pre commit + SRpcMsg rpcMsg; + syncEntry2OriginalRpc(pAppendEntry, &rpcMsg); + if (ths->pFsm != NULL) { + // if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->originalRpcType != TDMT_SYNC_NOOP) { + if (ths->pFsm->FpPreCommitCb != NULL && syncUtilUserPreCommit(pAppendEntry->originalRpcType)) { + SFsmCbMeta cbMeta = {0}; + cbMeta.index = pAppendEntry->index; + cbMeta.lastConfigIndex = syncNodeGetSnapshotConfigIndex(ths, cbMeta.index); + cbMeta.isWeak = pAppendEntry->isWeak; + cbMeta.code = 2; + cbMeta.state = ths->state; + cbMeta.seqNum = pAppendEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); + } + } + rpcFreeCont(rpcMsg.pCont); + } + + // free memory + syncEntryDestory(pExtraEntry); + syncEntryDestory(pAppendEntry); + + } else if (hasExtraEntries && !hasAppendEntries) { + // do nothing + + } else if (!hasExtraEntries && hasAppendEntries) { + SSyncRaftEntry* pAppendEntry = syncEntryDeserialize(pMsg->data, pMsg->dataLen); + if (pAppendEntry == NULL) { + syncNodeErrorLog(ths, "syncEntryDeserialize pAppendEntry2 error"); + return -1; + } + + // append new entries + ths->pLogStore->appendEntry(ths->pLogStore, pAppendEntry); + + // pre commit + SRpcMsg rpcMsg; + syncEntry2OriginalRpc(pAppendEntry, &rpcMsg); + if (ths->pFsm != NULL) { + // if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->originalRpcType != TDMT_SYNC_NOOP) { + if (ths->pFsm->FpPreCommitCb != NULL && syncUtilUserPreCommit(pAppendEntry->originalRpcType)) { + SFsmCbMeta cbMeta = {0}; + cbMeta.index = pAppendEntry->index; + cbMeta.lastConfigIndex = syncNodeGetSnapshotConfigIndex(ths, cbMeta.index); + cbMeta.isWeak = pAppendEntry->isWeak; + cbMeta.code = 3; + cbMeta.state = ths->state; + cbMeta.seqNum = pAppendEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); + } + } + rpcFreeCont(rpcMsg.pCont); + + // free memory + syncEntryDestory(pAppendEntry); + + } else if (!hasExtraEntries && !hasAppendEntries) { + // do nothing + + } else { + syncNodeLog3("", ths); + ASSERT(0); + } + + SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(ths->vgId); + pReply->srcId = ths->myRaftId; + pReply->destId = pMsg->srcId; + pReply->term = ths->pRaftStore->currentTerm; + pReply->success = true; + + if (hasAppendEntries) { + pReply->matchIndex = pMsg->prevLogIndex + 1; + } else { + pReply->matchIndex = pMsg->prevLogIndex; + } + + SRpcMsg rpcMsg; + syncAppendEntriesReply2RpcMsg(pReply, &rpcMsg); + syncNodeSendMsgById(&pReply->destId, ths, &rpcMsg); + syncAppendEntriesReplyDestroy(pReply); + + // maybe update commit index from leader + if (pMsg->commitIndex > ths->commitIndex) { + // has commit entry in local + if (pMsg->commitIndex <= ths->pLogStore->getLastIndex(ths->pLogStore)) { + SyncIndex beginIndex = ths->commitIndex + 1; + SyncIndex endIndex = pMsg->commitIndex; + + // update commit index + ths->commitIndex = pMsg->commitIndex; + + // call back Wal + ths->pLogStore->updateCommitIndex(ths->pLogStore, ths->commitIndex); + + int32_t code = syncNodeCommit(ths, beginIndex, endIndex, ths->state); + ASSERT(code == 0); + } + } + } + + return ret; +} + +#if 0 + int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { int32_t ret = 0; @@ -352,6 +626,8 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { return ret; } +#endif + static int32_t syncNodeMakeLogSame(SSyncNode* ths, SyncAppendEntries* pMsg) { int32_t code; diff --git a/source/libs/sync/src/syncAppendEntriesReply.c b/source/libs/sync/src/syncAppendEntriesReply.c index 6ca356b4f6049e602e698ca46802d49237bce32c..b7122b9c5233e683487985b6ff635b85252a2a58 100644 --- a/source/libs/sync/src/syncAppendEntriesReply.c +++ b/source/libs/sync/src/syncAppendEntriesReply.c @@ -40,6 +40,78 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* pMsg) { int32_t ret = 0; + // print log + syncAppendEntriesReplyLog2("==syncNodeOnAppendEntriesReplyCb==", pMsg); + + // if already drop replica, do not process + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { + syncNodeEventLog(ths, "recv sync-append-entries-reply, maybe replica already dropped"); + return 0; + } + + // drop stale response + if (pMsg->term < ths->pRaftStore->currentTerm) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "recv sync-append-entries-reply, recv-term:%lu, drop stale response", pMsg->term); + syncNodeEventLog(ths, logBuf); + return 0; + } + + if (gRaftDetailLog) { + syncNodeEventLog(ths, "recv sync-append-entries-reply, before"); + } + syncIndexMgrLog2("==syncNodeOnAppendEntriesReplyCb== before pNextIndex", ths->pNextIndex); + syncIndexMgrLog2("==syncNodeOnAppendEntriesReplyCb== before pMatchIndex", ths->pMatchIndex); + + // no need this code, because if I receive reply.term, then I must have sent for that term. + // if (pMsg->term > ths->pRaftStore->currentTerm) { + // syncNodeUpdateTerm(ths, pMsg->term); + // } + + if (pMsg->term > ths->pRaftStore->currentTerm) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "recv sync-append-entries-reply, error term, recv-term:%lu", pMsg->term); + syncNodeErrorLog(ths, logBuf); + return -1; + } + + ASSERT(pMsg->term == ths->pRaftStore->currentTerm); + + if (pMsg->success) { + // nextIndex' = [nextIndex EXCEPT ![i][j] = m.mmatchIndex + 1] + syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), pMsg->matchIndex + 1); + + // matchIndex' = [matchIndex EXCEPT ![i][j] = m.mmatchIndex] + syncIndexMgrSetIndex(ths->pMatchIndex, &(pMsg->srcId), pMsg->matchIndex); + + // maybe commit + syncMaybeAdvanceCommitIndex(ths); + + } else { + SyncIndex nextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId)); + + // notice! int64, uint64 + if (nextIndex > SYNC_INDEX_BEGIN) { + --nextIndex; + } else { + nextIndex = SYNC_INDEX_BEGIN; + } + syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), nextIndex); + } + + if (gRaftDetailLog) { + syncNodeEventLog(ths, "recv sync-append-entries-reply, after"); + } + syncIndexMgrLog2("==syncNodeOnAppendEntriesReplyCb== after pNextIndex", ths->pNextIndex); + syncIndexMgrLog2("==syncNodeOnAppendEntriesReplyCb== after pMatchIndex", ths->pMatchIndex); + + return ret; +} + +#if 0 +int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* pMsg) { + int32_t ret = 0; + char logBuf[128] = {0}; snprintf(logBuf, sizeof(logBuf), "==syncNodeOnAppendEntriesReplyCb== term:%lu", ths->pRaftStore->currentTerm); syncAppendEntriesReplyLog2(logBuf, pMsg); @@ -96,6 +168,7 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p return ret; } +#endif int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntriesReply* pMsg) { int32_t ret = 0; diff --git a/source/libs/sync/src/syncRequestVote.c b/source/libs/sync/src/syncRequestVote.c index 3270b5c0e4df72119ccf6e84006cbd85c8fcb1bc..95dec6cb831b884f4380ffee2097b0971a003702 100644 --- a/source/libs/sync/src/syncRequestVote.c +++ b/source/libs/sync/src/syncRequestVote.c @@ -45,6 +45,75 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { int32_t ret = 0; + syncRequestVoteLog2("==syncNodeOnRequestVoteCb==", pMsg); + + // if already drop replica, do not process + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { + do { + char logBuf[256]; + char host[64]; + uint16_t port; + syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); + snprintf(logBuf, sizeof(logBuf), + "recv sync-request-vote from %s:%d, term:%lu, lindex:%ld, lterm:%lu, maybe replica already dropped", + host, port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm); + syncNodeEventLog(ths, logBuf); + } while (0); + + return -1; + } + + // maybe update term + if (pMsg->term > ths->pRaftStore->currentTerm) { + syncNodeUpdateTerm(ths, pMsg->term); + } + ASSERT(pMsg->term <= ths->pRaftStore->currentTerm); + + bool logOK = (pMsg->lastLogTerm > ths->pLogStore->getLastTerm(ths->pLogStore)) || + ((pMsg->lastLogTerm == ths->pLogStore->getLastTerm(ths->pLogStore)) && + (pMsg->lastLogIndex >= ths->pLogStore->getLastIndex(ths->pLogStore))); + bool grant = (pMsg->term == ths->pRaftStore->currentTerm) && logOK && + ((!raftStoreHasVoted(ths->pRaftStore)) || (syncUtilSameId(&(ths->pRaftStore->voteFor), &(pMsg->srcId)))); + if (grant) { + // maybe has already voted for pMsg->srcId + // vote again, no harm + raftStoreVote(ths->pRaftStore, &(pMsg->srcId)); + + // forbid elect for this round + syncNodeResetElectTimer(ths); + } + + // send msg + SyncRequestVoteReply* pReply = syncRequestVoteReplyBuild(ths->vgId); + pReply->srcId = ths->myRaftId; + pReply->destId = pMsg->srcId; + pReply->term = ths->pRaftStore->currentTerm; + pReply->voteGranted = grant; + + // trace log + do { + char logBuf[256]; + char host[64]; + uint16_t port; + syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); + snprintf(logBuf, sizeof(logBuf), + "recv sync-request-vote from %s:%d, term:%lu, lindex:%ld, lterm:%lu, reply-grant:%d", host, port, + pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm, pReply->voteGranted); + syncNodeEventLog(ths, logBuf); + } while (0); + + SRpcMsg rpcMsg; + syncRequestVoteReply2RpcMsg(pReply, &rpcMsg); + syncNodeSendMsgById(&pReply->destId, ths, &rpcMsg); + syncRequestVoteReplyDestroy(pReply); + + return ret; +} + +#if 0 +int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { + int32_t ret = 0; + char logBuf[128] = {0}; snprintf(logBuf, sizeof(logBuf), "==syncNodeOnRequestVoteCb== term:%lu", ths->pRaftStore->currentTerm); syncRequestVoteLog2(logBuf, pMsg); @@ -81,6 +150,7 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { return ret; } +#endif static bool syncNodeOnRequestVoteLogOK(SSyncNode* pSyncNode, SyncRequestVote* pMsg) { SyncTerm myLastTerm = syncNodeGetLastTerm(pSyncNode); diff --git a/source/libs/sync/src/syncRequestVoteReply.c b/source/libs/sync/src/syncRequestVoteReply.c index 60963d0dd3b3bd4f7ce18de1a6746fd446306851..4c205d16ec1f20c51cf8efb62519a7b2d33370b5 100644 --- a/source/libs/sync/src/syncRequestVoteReply.c +++ b/source/libs/sync/src/syncRequestVoteReply.c @@ -40,6 +40,72 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) { int32_t ret = 0; + // print log + char logBuf[128] = {0}; + snprintf(logBuf, sizeof(logBuf), "==syncNodeOnRequestVoteReplyCb== term:%lu", ths->pRaftStore->currentTerm); + syncRequestVoteReplyLog2(logBuf, pMsg); + + // if already drop replica, do not process + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { + sInfo("recv SyncRequestVoteReply, maybe replica already dropped"); + return ret; + } + + // drop stale response + if (pMsg->term < ths->pRaftStore->currentTerm) { + sTrace("recv SyncRequestVoteReply, drop stale response, receive_term:%lu current_term:%lu", pMsg->term, + ths->pRaftStore->currentTerm); + return ret; + } + + // ASSERT(!(pMsg->term > ths->pRaftStore->currentTerm)); + // no need this code, because if I receive reply.term, then I must have sent for that term. + // if (pMsg->term > ths->pRaftStore->currentTerm) { + // syncNodeUpdateTerm(ths, pMsg->term); + // } + + if (pMsg->term > ths->pRaftStore->currentTerm) { + char logBuf[128] = {0}; + snprintf(logBuf, sizeof(logBuf), "syncNodeOnRequestVoteReplyCb error term, receive:%lu current:%lu", pMsg->term, + ths->pRaftStore->currentTerm); + syncNodePrint2(logBuf, ths); + sError("%s", logBuf); + return ret; + } + + ASSERT(pMsg->term == ths->pRaftStore->currentTerm); + + // This tallies votes even when the current state is not Candidate, + // but they won't be looked at, so it doesn't matter. + if (ths->state == TAOS_SYNC_STATE_CANDIDATE) { + votesRespondAdd(ths->pVotesRespond, pMsg); + if (pMsg->voteGranted) { + // add vote + voteGrantedVote(ths->pVotesGranted, pMsg); + + // maybe to leader + if (voteGrantedMajority(ths->pVotesGranted)) { + if (!ths->pVotesGranted->toLeader) { + syncNodeCandidate2Leader(ths); + + // prevent to leader again! + ths->pVotesGranted->toLeader = true; + } + } + } else { + ; + // do nothing + // UNCHANGED <> + } + } + + return ret; +} + +#if 0 +int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) { + int32_t ret = 0; + char logBuf[128] = {0}; snprintf(logBuf, sizeof(logBuf), "==syncNodeOnRequestVoteReplyCb== term:%lu", ths->pRaftStore->currentTerm); syncRequestVoteReplyLog2(logBuf, pMsg); @@ -93,6 +159,7 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) return ret; } +#endif int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) { int32_t ret = 0;