From e86ee68f8b85f84ac1c65ceb0fd030fddb5855a0 Mon Sep 17 00:00:00 2001 From: Minghao Li Date: Wed, 8 Jun 2022 22:43:58 +0800 Subject: [PATCH] refactor(sync): sync snapshot --- source/libs/sync/src/syncAppendEntries.c | 13 +++++-------- source/libs/sync/src/syncAppendEntriesReply.c | 3 ++- source/libs/sync/src/syncMain.c | 4 ++-- source/libs/sync/src/syncRequestVote.c | 3 ++- source/libs/sync/src/syncRequestVoteReply.c | 3 ++- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/source/libs/sync/src/syncAppendEntries.c b/source/libs/sync/src/syncAppendEntries.c index 7f654975bb..170b50cfed 100644 --- a/source/libs/sync/src/syncAppendEntries.c +++ b/source/libs/sync/src/syncAppendEntries.c @@ -534,14 +534,11 @@ int32_t syncNodeOnAppendEntriesSnapshotCb(SSyncNode* ths, SyncAppendEntries* pMs ths->pRaftStore->currentTerm); syncAppendEntriesLog2(logBuf, pMsg); - // if I am standby, to be added into a raft group, I should process SyncAppendEntries msg - /* - // if already drop replica, do not process - if (!syncNodeInRaftGroup(ths, &(pMsg->srcId))) { - sInfo("recv SyncAppendEntries maybe replica already dropped"); - return ret; - } - */ + // if already drop replica, do not process + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { + sInfo("recv SyncAppendEntries maybe replica already dropped"); + return ret; + } // maybe update term if (pMsg->term > ths->pRaftStore->currentTerm) { diff --git a/source/libs/sync/src/syncAppendEntriesReply.c b/source/libs/sync/src/syncAppendEntriesReply.c index 947116f965..150b6f2410 100644 --- a/source/libs/sync/src/syncAppendEntriesReply.c +++ b/source/libs/sync/src/syncAppendEntriesReply.c @@ -17,6 +17,7 @@ #include "syncCommit.h" #include "syncIndexMgr.h" #include "syncInt.h" +#include "syncRaftCfg.h" #include "syncRaftLog.h" #include "syncRaftStore.h" #include "syncSnapshot.h" @@ -106,7 +107,7 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries syncAppendEntriesReplyLog2(logBuf, pMsg); // if already drop replica, do not process - if (!syncNodeInRaftGroup(ths, &(pMsg->srcId))) { + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { sInfo("recv SyncAppendEntriesReply, maybe replica already dropped"); return ret; } diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index acbe658914..6287172e76 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -1039,10 +1039,10 @@ char* syncNode2SimpleStr(const SSyncNode* pSyncNode) { "syncNode2SimpleStr vgId:%d currentTerm:%lu, commitIndex:%ld, state:%d %s, isStandBy:%d, " "electTimerLogicClock:%lu, " "electTimerLogicClockUser:%lu, " - "electTimerMS:%d", + "electTimerMS:%d, replicaNum:%d", pSyncNode->vgId, pSyncNode->pRaftStore->currentTerm, pSyncNode->commitIndex, pSyncNode->state, syncUtilState2String(pSyncNode->state), pSyncNode->pRaftCfg->isStandBy, pSyncNode->electTimerLogicClock, - pSyncNode->electTimerLogicClockUser, pSyncNode->electTimerMS); + pSyncNode->electTimerLogicClockUser, pSyncNode->electTimerMS, pSyncNode->replicaNum); return s; } diff --git a/source/libs/sync/src/syncRequestVote.c b/source/libs/sync/src/syncRequestVote.c index d939244906..9ed7f00982 100644 --- a/source/libs/sync/src/syncRequestVote.c +++ b/source/libs/sync/src/syncRequestVote.c @@ -15,6 +15,7 @@ #include "syncRequestVote.h" #include "syncInt.h" +#include "syncRaftCfg.h" #include "syncRaftStore.h" #include "syncUtil.h" #include "syncVoteMgr.h" @@ -104,7 +105,7 @@ int32_t syncNodeOnRequestVoteSnapshotCb(SSyncNode* ths, SyncRequestVote* pMsg) { syncRequestVoteLog2(logBuf, pMsg); // if already drop replica, do not process - if (!syncNodeInRaftGroup(ths, &(pMsg->srcId))) { + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { sInfo("recv SyncRequestVote maybe replica already dropped"); return ret; } diff --git a/source/libs/sync/src/syncRequestVoteReply.c b/source/libs/sync/src/syncRequestVoteReply.c index 9cc905d847..5d041cefcd 100644 --- a/source/libs/sync/src/syncRequestVoteReply.c +++ b/source/libs/sync/src/syncRequestVoteReply.c @@ -15,6 +15,7 @@ #include "syncRequestVoteReply.h" #include "syncInt.h" +#include "syncRaftCfg.h" #include "syncRaftStore.h" #include "syncUtil.h" #include "syncVoteMgr.h" @@ -102,7 +103,7 @@ int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteRepl syncRequestVoteReplyLog2(logBuf, pMsg); // if already drop replica, do not process - if (!syncNodeInRaftGroup(ths, &(pMsg->srcId))) { + if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { sInfo("recv SyncRequestVoteReply, maybe replica already dropped"); return ret; }