refactor(sync): add pre snapshot
This commit is contained in:
parent
59448cd18e
commit
f7191d2ffd
|
@ -22,9 +22,8 @@ extern "C" {
|
||||||
|
|
||||||
#include "sync.h"
|
#include "sync.h"
|
||||||
#include "syncTools.h"
|
#include "syncTools.h"
|
||||||
#include "tlog.h"
|
|
||||||
#include "ttimer.h"
|
|
||||||
#include "taosdef.h"
|
#include "taosdef.h"
|
||||||
|
#include "tlog.h"
|
||||||
#include "ttimer.h"
|
#include "ttimer.h"
|
||||||
|
|
||||||
// clang-format off
|
// clang-format off
|
||||||
|
@ -344,6 +343,12 @@ void syncLogRecvSyncPreSnapshot(SSyncNode* pSyncNode, const SyncPreSnapshot* pMs
|
||||||
void syncLogSendSyncPreSnapshotReply(SSyncNode* pSyncNode, const SyncPreSnapshotReply* pMsg, const char* s);
|
void syncLogSendSyncPreSnapshotReply(SSyncNode* pSyncNode, const SyncPreSnapshotReply* pMsg, const char* s);
|
||||||
void syncLogRecvSyncPreSnapshotReply(SSyncNode* pSyncNode, const SyncPreSnapshotReply* pMsg, const char* s);
|
void syncLogRecvSyncPreSnapshotReply(SSyncNode* pSyncNode, const SyncPreSnapshotReply* pMsg, const char* s);
|
||||||
|
|
||||||
|
void syncLogSendSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s);
|
||||||
|
void syncLogRecvSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s);
|
||||||
|
|
||||||
|
void syncLogSendSyncSnapshotRsp(SSyncNode* pSyncNode, const SyncSnapshotRsp* pMsg, const char* s);
|
||||||
|
void syncLogRecvSyncSnapshotRsp(SSyncNode* pSyncNode, const SyncSnapshotRsp* pMsg, const char* s);
|
||||||
|
|
||||||
// for debug --------------
|
// for debug --------------
|
||||||
void syncNodePrint(SSyncNode* pObj);
|
void syncNodePrint(SSyncNode* pObj);
|
||||||
void syncNodePrint2(char* s, SSyncNode* pObj);
|
void syncNodePrint2(char* s, SSyncNode* pObj);
|
||||||
|
|
|
@ -28,10 +28,11 @@ extern "C" {
|
||||||
#include "syncMessage.h"
|
#include "syncMessage.h"
|
||||||
#include "taosdef.h"
|
#include "taosdef.h"
|
||||||
|
|
||||||
#define SYNC_SNAPSHOT_SEQ_INVALID -1
|
#define SYNC_SNAPSHOT_SEQ_INVALID -2
|
||||||
#define SYNC_SNAPSHOT_SEQ_FORCE_CLOSE -2
|
#define SYNC_SNAPSHOT_SEQ_FORCE_CLOSE -3
|
||||||
#define SYNC_SNAPSHOT_SEQ_BEGIN 0
|
#define SYNC_SNAPSHOT_SEQ_PRE_SNAPSHOT -1
|
||||||
#define SYNC_SNAPSHOT_SEQ_END 0x7FFFFFFF
|
#define SYNC_SNAPSHOT_SEQ_BEGIN 0
|
||||||
|
#define SYNC_SNAPSHOT_SEQ_END 0x7FFFFFFF
|
||||||
|
|
||||||
#define SYNC_SNAPSHOT_RETRY_MS 5000
|
#define SYNC_SNAPSHOT_RETRY_MS 5000
|
||||||
|
|
||||||
|
@ -47,19 +48,19 @@ typedef struct SSyncSnapshotSender {
|
||||||
SSnapshot snapshot;
|
SSnapshot snapshot;
|
||||||
SSyncCfg lastConfig;
|
SSyncCfg lastConfig;
|
||||||
int64_t sendingMS;
|
int64_t sendingMS;
|
||||||
SSyncNode *pSyncNode;
|
|
||||||
int32_t replicaIndex;
|
|
||||||
SyncTerm term;
|
SyncTerm term;
|
||||||
SyncTerm privateTerm;
|
|
||||||
int64_t startTime;
|
int64_t startTime;
|
||||||
bool finish;
|
bool finish;
|
||||||
|
|
||||||
|
// init when create
|
||||||
|
SSyncNode *pSyncNode;
|
||||||
|
int32_t replicaIndex;
|
||||||
} SSyncSnapshotSender;
|
} SSyncSnapshotSender;
|
||||||
|
|
||||||
SSyncSnapshotSender *snapshotSenderCreate(SSyncNode *pSyncNode, int32_t replicaIndex);
|
SSyncSnapshotSender *snapshotSenderCreate(SSyncNode *pSyncNode, int32_t replicaIndex);
|
||||||
void snapshotSenderDestroy(SSyncSnapshotSender *pSender);
|
void snapshotSenderDestroy(SSyncSnapshotSender *pSender);
|
||||||
bool snapshotSenderIsStart(SSyncSnapshotSender *pSender);
|
bool snapshotSenderIsStart(SSyncSnapshotSender *pSender);
|
||||||
int32_t snapshotSenderStart(SSyncSnapshotSender *pSender, SSnapshotParam snapshotParam, SSnapshot snapshot,
|
int32_t snapshotSenderStart(SSyncSnapshotSender *pSender);
|
||||||
void *pReader);
|
|
||||||
int32_t snapshotSenderStop(SSyncSnapshotSender *pSender, bool finish);
|
int32_t snapshotSenderStop(SSyncSnapshotSender *pSender, bool finish);
|
||||||
int32_t snapshotSend(SSyncSnapshotSender *pSender);
|
int32_t snapshotSend(SSyncSnapshotSender *pSender);
|
||||||
int32_t snapshotReSend(SSyncSnapshotSender *pSender);
|
int32_t snapshotReSend(SSyncSnapshotSender *pSender);
|
||||||
|
@ -76,11 +77,13 @@ typedef struct SSyncSnapshotReceiver {
|
||||||
int32_t ack;
|
int32_t ack;
|
||||||
void *pWriter;
|
void *pWriter;
|
||||||
SyncTerm term;
|
SyncTerm term;
|
||||||
SyncTerm privateTerm;
|
|
||||||
SSnapshotParam snapshotParam;
|
SSnapshotParam snapshotParam;
|
||||||
SSnapshot snapshot;
|
SSnapshot snapshot;
|
||||||
SRaftId fromId;
|
SRaftId fromId;
|
||||||
SSyncNode *pSyncNode;
|
int64_t startTime;
|
||||||
|
|
||||||
|
// init when create
|
||||||
|
SSyncNode *pSyncNode;
|
||||||
|
|
||||||
} SSyncSnapshotReceiver;
|
} SSyncSnapshotReceiver;
|
||||||
|
|
||||||
|
|
|
@ -142,7 +142,6 @@ int32_t syncNodeOnAppendEntries(SSyncNode* ths, SyncAppendEntries* pMsg) {
|
||||||
// pReply->matchIndex = ths->pLogStore->syncLogLastIndex(ths->pLogStore);
|
// pReply->matchIndex = ths->pLogStore->syncLogLastIndex(ths->pLogStore);
|
||||||
pReply->matchIndex = SYNC_INDEX_INVALID;
|
pReply->matchIndex = SYNC_INDEX_INVALID;
|
||||||
pReply->lastSendIndex = pMsg->prevLogIndex + 1;
|
pReply->lastSendIndex = pMsg->prevLogIndex + 1;
|
||||||
pReply->privateTerm = ths->pNewNodeReceiver->privateTerm;
|
|
||||||
pReply->startTime = ths->startTime;
|
pReply->startTime = ths->startTime;
|
||||||
|
|
||||||
if (pMsg->term < ths->pRaftStore->currentTerm) {
|
if (pMsg->term < ths->pRaftStore->currentTerm) {
|
||||||
|
|
|
@ -73,6 +73,7 @@ static void syncNodeStartSnapshotOnce(SSyncNode* ths, SyncIndex beginIndex, Sync
|
||||||
int32_t code = ths->pFsm->FpSnapshotStartRead(ths->pFsm, &readerParam, &pReader);
|
int32_t code = ths->pFsm->FpSnapshotStartRead(ths->pFsm, &readerParam, &pReader);
|
||||||
ASSERT(code == 0);
|
ASSERT(code == 0);
|
||||||
|
|
||||||
|
#if 0
|
||||||
if (pMsg->privateTerm < pSender->privateTerm) {
|
if (pMsg->privateTerm < pSender->privateTerm) {
|
||||||
ASSERT(pReader != NULL);
|
ASSERT(pReader != NULL);
|
||||||
snapshotSenderStart(pSender, readerParam, snapshot, pReader);
|
snapshotSenderStart(pSender, readerParam, snapshot, pReader);
|
||||||
|
@ -82,6 +83,7 @@ static void syncNodeStartSnapshotOnce(SSyncNode* ths, SyncIndex beginIndex, Sync
|
||||||
ths->pFsm->FpSnapshotStopRead(ths->pFsm, pReader);
|
ths->pFsm->FpSnapshotStopRead(ths->pFsm, pReader);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
int32_t syncNodeOnAppendEntriesReply(SSyncNode* ths, SyncAppendEntriesReply* pMsg) {
|
int32_t syncNodeOnAppendEntriesReply(SSyncNode* ths, SyncAppendEntriesReply* pMsg) {
|
||||||
|
|
|
@ -2355,6 +2355,7 @@ void syncNodeBecomeLeader(SSyncNode* pSyncNode, const char* debugStr) {
|
||||||
// init peer mgr
|
// init peer mgr
|
||||||
syncNodePeerStateInit(pSyncNode);
|
syncNodePeerStateInit(pSyncNode);
|
||||||
|
|
||||||
|
#if 0
|
||||||
// update sender private term
|
// update sender private term
|
||||||
SSyncSnapshotSender* pMySender = syncNodeGetSnapshotSender(pSyncNode, &(pSyncNode->myRaftId));
|
SSyncSnapshotSender* pMySender = syncNodeGetSnapshotSender(pSyncNode, &(pSyncNode->myRaftId));
|
||||||
if (pMySender != NULL) {
|
if (pMySender != NULL) {
|
||||||
|
@ -2365,6 +2366,7 @@ void syncNodeBecomeLeader(SSyncNode* pSyncNode, const char* debugStr) {
|
||||||
}
|
}
|
||||||
(pMySender->privateTerm) += 100;
|
(pMySender->privateTerm) += 100;
|
||||||
}
|
}
|
||||||
|
#endif
|
||||||
|
|
||||||
// close receiver
|
// close receiver
|
||||||
if (snapshotReceiverIsStart(pSyncNode->pNewNodeReceiver)) {
|
if (snapshotReceiverIsStart(pSyncNode->pNewNodeReceiver)) {
|
||||||
|
@ -3718,3 +3720,11 @@ void syncLogRecvSyncPreSnapshotReply(SSyncNode* pSyncNode, const SyncPreSnapshot
|
||||||
pMsg->term, pMsg->snapStart, s);
|
pMsg->term, pMsg->snapStart, s);
|
||||||
syncNodeEventLog(pSyncNode, logBuf);
|
syncNodeEventLog(pSyncNode, logBuf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void syncLogSendSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s) {}
|
||||||
|
|
||||||
|
void syncLogRecvSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s) {}
|
||||||
|
|
||||||
|
void syncLogSendSyncSnapshotRsp(SSyncNode* pSyncNode, const SyncSnapshotRsp* pMsg, const char* s) {}
|
||||||
|
|
||||||
|
void syncLogRecvSyncSnapshotRsp(SSyncNode* pSyncNode, const SyncSnapshotRsp* pMsg, const char* s) {}
|
||||||
|
|
|
@ -2873,8 +2873,8 @@ cJSON* syncSnapshotSend2Json(const SyncSnapshotSend* pMsg) {
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term);
|
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term);
|
||||||
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
||||||
|
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->privateTerm);
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->startTime);
|
||||||
cJSON_AddStringToObject(pRoot, "privateTerm", u64buf);
|
cJSON_AddStringToObject(pRoot, "startTime", u64buf);
|
||||||
|
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->beginIndex);
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->beginIndex);
|
||||||
cJSON_AddStringToObject(pRoot, "beginIndex", u64buf);
|
cJSON_AddStringToObject(pRoot, "beginIndex", u64buf);
|
||||||
|
@ -3048,8 +3048,8 @@ cJSON* syncSnapshotRsp2Json(const SyncSnapshotRsp* pMsg) {
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term);
|
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term);
|
||||||
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
||||||
|
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->privateTerm);
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->startTime);
|
||||||
cJSON_AddStringToObject(pRoot, "privateTerm", u64buf);
|
cJSON_AddStringToObject(pRoot, "startTime", u64buf);
|
||||||
|
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastIndex);
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastIndex);
|
||||||
cJSON_AddStringToObject(pRoot, "lastIndex", u64buf);
|
cJSON_AddStringToObject(pRoot, "lastIndex", u64buf);
|
||||||
|
@ -3059,6 +3059,9 @@ cJSON* syncSnapshotRsp2Json(const SyncSnapshotRsp* pMsg) {
|
||||||
|
|
||||||
cJSON_AddNumberToObject(pRoot, "ack", pMsg->ack);
|
cJSON_AddNumberToObject(pRoot, "ack", pMsg->ack);
|
||||||
cJSON_AddNumberToObject(pRoot, "code", pMsg->code);
|
cJSON_AddNumberToObject(pRoot, "code", pMsg->code);
|
||||||
|
|
||||||
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->snapBeginIndex);
|
||||||
|
cJSON_AddStringToObject(pRoot, "snap-begin", u64buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
cJSON* pJson = cJSON_CreateObject();
|
cJSON* pJson = cJSON_CreateObject();
|
||||||
|
|
|
@ -62,8 +62,8 @@ int32_t syncNodeReplicateOne(SSyncNode* pSyncNode, SRaftId* pDestId) {
|
||||||
syncNodeEventLog(pSyncNode, logBuf);
|
syncNodeEventLog(pSyncNode, logBuf);
|
||||||
|
|
||||||
// start snapshot
|
// start snapshot
|
||||||
int32_t code = syncNodeStartSnapshot(pSyncNode, pDestId);
|
// int32_t code = syncNodeStartSnapshot(pSyncNode, pDestId);
|
||||||
ASSERT(code == 0);
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -41,8 +41,6 @@ SSyncSnapshotSender *snapshotSenderCreate(SSyncNode *pSyncNode, int32_t replicaI
|
||||||
}
|
}
|
||||||
memset(pSender, 0, sizeof(*pSender));
|
memset(pSender, 0, sizeof(*pSender));
|
||||||
|
|
||||||
int64_t timeNow = taosGetTimestampMs();
|
|
||||||
|
|
||||||
pSender->start = false;
|
pSender->start = false;
|
||||||
pSender->seq = SYNC_SNAPSHOT_SEQ_INVALID;
|
pSender->seq = SYNC_SNAPSHOT_SEQ_INVALID;
|
||||||
pSender->ack = SYNC_SNAPSHOT_SEQ_INVALID;
|
pSender->ack = SYNC_SNAPSHOT_SEQ_INVALID;
|
||||||
|
@ -53,8 +51,7 @@ SSyncSnapshotSender *snapshotSenderCreate(SSyncNode *pSyncNode, int32_t replicaI
|
||||||
pSender->pSyncNode = pSyncNode;
|
pSender->pSyncNode = pSyncNode;
|
||||||
pSender->replicaIndex = replicaIndex;
|
pSender->replicaIndex = replicaIndex;
|
||||||
pSender->term = pSyncNode->pRaftStore->currentTerm;
|
pSender->term = pSyncNode->pRaftStore->currentTerm;
|
||||||
pSender->privateTerm = timeNow + 100;
|
pSender->startTime = 0;
|
||||||
pSender->startTime = timeNow;
|
|
||||||
pSender->pSyncNode->pFsm->FpGetSnapshotInfo(pSender->pSyncNode->pFsm, &(pSender->snapshot));
|
pSender->pSyncNode->pFsm->FpGetSnapshotInfo(pSender->pSyncNode->pFsm, &(pSender->snapshot));
|
||||||
pSender->finish = false;
|
pSender->finish = false;
|
||||||
} else {
|
} else {
|
||||||
|
@ -88,88 +85,30 @@ void snapshotSenderDestroy(SSyncSnapshotSender *pSender) {
|
||||||
|
|
||||||
bool snapshotSenderIsStart(SSyncSnapshotSender *pSender) { return pSender->start; }
|
bool snapshotSenderIsStart(SSyncSnapshotSender *pSender) { return pSender->start; }
|
||||||
|
|
||||||
// begin send snapshot by param, snapshot, pReader
|
int32_t snapshotSenderStart(SSyncSnapshotSender *pSender) {
|
||||||
//
|
|
||||||
// action:
|
|
||||||
// 1. assert reader not start
|
|
||||||
// 2. update state
|
|
||||||
// 3. send first snapshot block
|
|
||||||
int32_t snapshotSenderStart(SSyncSnapshotSender *pSender, SSnapshotParam snapshotParam, SSnapshot snapshot,
|
|
||||||
void *pReader) {
|
|
||||||
ASSERT(!snapshotSenderIsStart(pSender));
|
ASSERT(!snapshotSenderIsStart(pSender));
|
||||||
|
|
||||||
// init snapshot, parm, reader
|
|
||||||
ASSERT(pSender->pReader == NULL);
|
|
||||||
pSender->pReader = pReader;
|
|
||||||
pSender->snapshot = snapshot;
|
|
||||||
pSender->snapshotParam = snapshotParam;
|
|
||||||
|
|
||||||
// init current block
|
|
||||||
if (pSender->pCurrentBlock != NULL) {
|
|
||||||
taosMemoryFree(pSender->pCurrentBlock);
|
|
||||||
}
|
|
||||||
pSender->blockLen = 0;
|
|
||||||
|
|
||||||
// update term
|
|
||||||
pSender->term = pSender->pSyncNode->pRaftStore->currentTerm;
|
|
||||||
++(pSender->privateTerm); // increase private term
|
|
||||||
|
|
||||||
// update state
|
|
||||||
pSender->finish = false;
|
|
||||||
pSender->start = true;
|
pSender->start = true;
|
||||||
pSender->seq = SYNC_SNAPSHOT_SEQ_BEGIN;
|
pSender->seq = SYNC_SNAPSHOT_SEQ_BEGIN;
|
||||||
pSender->ack = SYNC_SNAPSHOT_SEQ_INVALID;
|
pSender->ack = SYNC_SNAPSHOT_SEQ_INVALID;
|
||||||
|
pSender->pReader = NULL;
|
||||||
|
pSender->pCurrentBlock = NULL;
|
||||||
|
pSender->blockLen = 0;
|
||||||
|
|
||||||
// init last config
|
pSender->snapshotParam.start = SYNC_INDEX_INVALID;
|
||||||
if (pSender->snapshot.lastConfigIndex != SYNC_INDEX_INVALID) {
|
pSender->snapshotParam.end = SYNC_INDEX_INVALID;
|
||||||
int32_t code = 0;
|
|
||||||
SSyncRaftEntry *pEntry = NULL;
|
|
||||||
bool getLastConfig = false;
|
|
||||||
|
|
||||||
code = pSender->pSyncNode->pLogStore->syncLogGetEntry(pSender->pSyncNode->pLogStore,
|
pSender->snapshot.data = NULL;
|
||||||
pSender->snapshot.lastConfigIndex, &pEntry);
|
pSender->snapshotParam.end = SYNC_INDEX_INVALID;
|
||||||
if (code == 0 && pEntry != NULL) {
|
pSender->snapshot.lastApplyIndex = SYNC_INDEX_INVALID;
|
||||||
SRpcMsg rpcMsg;
|
pSender->snapshot.lastApplyTerm = SYNC_TERM_INVALID;
|
||||||
syncEntry2OriginalRpc(pEntry, &rpcMsg);
|
pSender->snapshot.lastConfigIndex = SYNC_INDEX_INVALID;
|
||||||
|
|
||||||
SSyncCfg lastConfig;
|
memset(&(pSender->lastConfig), 0, sizeof(pSender->lastConfig));
|
||||||
int32_t ret = syncCfgFromStr(rpcMsg.pCont, &lastConfig);
|
pSender->sendingMS = 0;
|
||||||
ASSERT(ret == 0);
|
pSender->term = pSender->pSyncNode->pRaftStore->currentTerm;
|
||||||
pSender->lastConfig = lastConfig;
|
pSender->startTime = taosGetTimestampMs();
|
||||||
getLastConfig = true;
|
pSender->finish = false;
|
||||||
|
|
||||||
rpcFreeCont(rpcMsg.pCont);
|
|
||||||
syncEntryDestory(pEntry);
|
|
||||||
} else {
|
|
||||||
if (pSender->snapshot.lastConfigIndex == pSender->pSyncNode->pRaftCfg->lastConfigIndex) {
|
|
||||||
sTrace("vgId:%d, sync sender get cfg from local", pSender->pSyncNode->vgId);
|
|
||||||
pSender->lastConfig = pSender->pSyncNode->pRaftCfg->cfg;
|
|
||||||
getLastConfig = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// last config not found in wal, update to -1
|
|
||||||
if (!getLastConfig) {
|
|
||||||
SyncIndex oldLastConfigIndex = pSender->snapshot.lastConfigIndex;
|
|
||||||
SyncIndex newLastConfigIndex = SYNC_INDEX_INVALID;
|
|
||||||
pSender->snapshot.lastConfigIndex = SYNC_INDEX_INVALID;
|
|
||||||
memset(&(pSender->lastConfig), 0, sizeof(SSyncCfg));
|
|
||||||
|
|
||||||
// event log
|
|
||||||
do {
|
|
||||||
char logBuf[128];
|
|
||||||
snprintf(logBuf, sizeof(logBuf), "snapshot sender update lcindex from %" PRId64 " to %" PRId64,
|
|
||||||
oldLastConfigIndex, newLastConfigIndex);
|
|
||||||
char *eventLog = snapshotSender2SimpleStr(pSender, logBuf);
|
|
||||||
syncNodeEventLog(pSender->pSyncNode, eventLog);
|
|
||||||
taosMemoryFree(eventLog);
|
|
||||||
} while (0);
|
|
||||||
}
|
|
||||||
|
|
||||||
} else {
|
|
||||||
// no last config
|
|
||||||
memset(&(pSender->lastConfig), 0, sizeof(SSyncCfg));
|
|
||||||
}
|
|
||||||
|
|
||||||
// build begin msg
|
// build begin msg
|
||||||
SyncSnapshotSend *pMsg = syncSnapshotSendBuild(0, pSender->pSyncNode->vgId);
|
SyncSnapshotSend *pMsg = syncSnapshotSendBuild(0, pSender->pSyncNode->vgId);
|
||||||
|
@ -181,8 +120,8 @@ int32_t snapshotSenderStart(SSyncSnapshotSender *pSender, SSnapshotParam snapsho
|
||||||
pMsg->lastTerm = pSender->snapshot.lastApplyTerm;
|
pMsg->lastTerm = pSender->snapshot.lastApplyTerm;
|
||||||
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
||||||
pMsg->lastConfig = pSender->lastConfig;
|
pMsg->lastConfig = pSender->lastConfig;
|
||||||
pMsg->seq = pSender->seq; // SYNC_SNAPSHOT_SEQ_BEGIN
|
pMsg->startTime = pSender->startTime;
|
||||||
pMsg->privateTerm = pSender->privateTerm;
|
pMsg->seq = SYNC_SNAPSHOT_SEQ_PRE_SNAPSHOT;
|
||||||
|
|
||||||
// send msg
|
// send msg
|
||||||
SRpcMsg rpcMsg;
|
SRpcMsg rpcMsg;
|
||||||
|
@ -201,6 +140,10 @@ int32_t snapshotSenderStart(SSyncSnapshotSender *pSender, SSnapshotParam snapsho
|
||||||
}
|
}
|
||||||
|
|
||||||
int32_t snapshotSenderStop(SSyncSnapshotSender *pSender, bool finish) {
|
int32_t snapshotSenderStop(SSyncSnapshotSender *pSender, bool finish) {
|
||||||
|
// update flag
|
||||||
|
pSender->start = false;
|
||||||
|
pSender->finish = finish;
|
||||||
|
|
||||||
// close reader
|
// close reader
|
||||||
if (pSender->pReader != NULL) {
|
if (pSender->pReader != NULL) {
|
||||||
int32_t ret = pSender->pSyncNode->pFsm->FpSnapshotStopRead(pSender->pSyncNode->pFsm, pSender->pReader);
|
int32_t ret = pSender->pSyncNode->pFsm->FpSnapshotStopRead(pSender->pSyncNode->pFsm, pSender->pReader);
|
||||||
|
@ -215,12 +158,6 @@ int32_t snapshotSenderStop(SSyncSnapshotSender *pSender, bool finish) {
|
||||||
pSender->blockLen = 0;
|
pSender->blockLen = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
// update flag
|
|
||||||
pSender->start = false;
|
|
||||||
pSender->finish = finish;
|
|
||||||
|
|
||||||
// do not update term, maybe print
|
|
||||||
|
|
||||||
// event log
|
// event log
|
||||||
do {
|
do {
|
||||||
char *eventLog = snapshotSender2SimpleStr(pSender, "snapshot sender stop");
|
char *eventLog = snapshotSender2SimpleStr(pSender, "snapshot sender stop");
|
||||||
|
@ -263,7 +200,9 @@ int32_t snapshotSend(SSyncSnapshotSender *pSender) {
|
||||||
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
||||||
pMsg->lastConfig = pSender->lastConfig;
|
pMsg->lastConfig = pSender->lastConfig;
|
||||||
pMsg->seq = pSender->seq;
|
pMsg->seq = pSender->seq;
|
||||||
pMsg->privateTerm = pSender->privateTerm;
|
|
||||||
|
// pMsg->privateTerm = pSender->privateTerm;
|
||||||
|
|
||||||
memcpy(pMsg->data, pSender->pCurrentBlock, pSender->blockLen);
|
memcpy(pMsg->data, pSender->pCurrentBlock, pSender->blockLen);
|
||||||
|
|
||||||
// send msg
|
// send msg
|
||||||
|
@ -302,7 +241,9 @@ int32_t snapshotReSend(SSyncSnapshotSender *pSender) {
|
||||||
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
pMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
||||||
pMsg->lastConfig = pSender->lastConfig;
|
pMsg->lastConfig = pSender->lastConfig;
|
||||||
pMsg->seq = pSender->seq;
|
pMsg->seq = pSender->seq;
|
||||||
pMsg->privateTerm = pSender->privateTerm;
|
|
||||||
|
// pMsg->privateTerm = pSender->privateTerm;
|
||||||
|
|
||||||
memcpy(pMsg->data, pSender->pCurrentBlock, pSender->blockLen);
|
memcpy(pMsg->data, pSender->pCurrentBlock, pSender->blockLen);
|
||||||
|
|
||||||
// send msg
|
// send msg
|
||||||
|
@ -367,8 +308,10 @@ cJSON *snapshotSender2Json(SSyncSnapshotSender *pSender) {
|
||||||
cJSON_AddNumberToObject(pRoot, "replicaIndex", pSender->replicaIndex);
|
cJSON_AddNumberToObject(pRoot, "replicaIndex", pSender->replicaIndex);
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSender->term);
|
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSender->term);
|
||||||
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSender->privateTerm);
|
|
||||||
cJSON_AddStringToObject(pRoot, "privateTerm", u64buf);
|
// snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSender->privateTerm);
|
||||||
|
// cJSON_AddStringToObject(pRoot, "privateTerm", u64buf);
|
||||||
|
|
||||||
cJSON_AddNumberToObject(pRoot, "finish", pSender->finish);
|
cJSON_AddNumberToObject(pRoot, "finish", pSender->finish);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -395,30 +338,38 @@ char *snapshotSender2SimpleStr(SSyncSnapshotSender *pSender, char *event) {
|
||||||
|
|
||||||
snprintf(s, len,
|
snprintf(s, len,
|
||||||
"%s {%p s-param:%" PRId64 " e-param:%" PRId64 " laindex:%" PRId64 " laterm:%" PRIu64 " lcindex:%" PRId64
|
"%s {%p s-param:%" PRId64 " e-param:%" PRId64 " laindex:%" PRId64 " laterm:%" PRIu64 " lcindex:%" PRId64
|
||||||
" seq:%d ack:%d finish:%d pterm:%" PRIu64
|
" seq:%d ack:%d finish:%d replica-index:%d %s:%d}",
|
||||||
" "
|
|
||||||
"replica-index:%d %s:%d}",
|
|
||||||
event, pSender, pSender->snapshotParam.start, pSender->snapshotParam.end, pSender->snapshot.lastApplyIndex,
|
event, pSender, pSender->snapshotParam.start, pSender->snapshotParam.end, pSender->snapshot.lastApplyIndex,
|
||||||
pSender->snapshot.lastApplyTerm, pSender->snapshot.lastConfigIndex, pSender->seq, pSender->ack,
|
pSender->snapshot.lastApplyTerm, pSender->snapshot.lastConfigIndex, pSender->seq, pSender->ack,
|
||||||
pSender->finish, pSender->privateTerm, pSender->replicaIndex, host, port);
|
pSender->finish, pSender->replicaIndex, host, port);
|
||||||
|
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
int32_t syncNodeStartSnapshot(SSyncNode *pSyncNode, SRaftId *pDestId) {
|
int32_t syncNodeStartSnapshot(SSyncNode *pSyncNode, SRaftId *pDestId) {
|
||||||
// calculate <start, end> index
|
syncNodeEventLog(pSyncNode, "starting snapshot ...");
|
||||||
|
|
||||||
syncNodeEventLog(pSyncNode, "start snapshot ...");
|
|
||||||
|
|
||||||
SSyncSnapshotSender *pSender = syncNodeGetSnapshotSender(pSyncNode, pDestId);
|
SSyncSnapshotSender *pSender = syncNodeGetSnapshotSender(pSyncNode, pDestId);
|
||||||
if (pSender == NULL) {
|
if (pSender == NULL) {
|
||||||
// create sender
|
syncNodeErrorLog(pSyncNode, "start snapshot error, sender is null");
|
||||||
} else {
|
return -1;
|
||||||
// if <start, end> is same
|
|
||||||
// return 0;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// send begin msg
|
int32_t code = 0;
|
||||||
|
|
||||||
|
if (snapshotSenderIsStart(pSender)) {
|
||||||
|
code = snapshotSenderStop(pSender, false);
|
||||||
|
if (code != 0) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "snapshot sender stop error");
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
code = snapshotSenderStart(pSender);
|
||||||
|
if (code != 0) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "snapshot sender start error");
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -440,7 +391,6 @@ SSyncSnapshotReceiver *snapshotReceiverCreate(SSyncNode *pSyncNode, SRaftId from
|
||||||
pReceiver->pSyncNode = pSyncNode;
|
pReceiver->pSyncNode = pSyncNode;
|
||||||
pReceiver->fromId = fromId;
|
pReceiver->fromId = fromId;
|
||||||
pReceiver->term = pSyncNode->pRaftStore->currentTerm;
|
pReceiver->term = pSyncNode->pRaftStore->currentTerm;
|
||||||
pReceiver->privateTerm = 0;
|
|
||||||
pReceiver->snapshot.data = NULL;
|
pReceiver->snapshot.data = NULL;
|
||||||
pReceiver->snapshot.lastApplyIndex = SYNC_INDEX_INVALID;
|
pReceiver->snapshot.lastApplyIndex = SYNC_INDEX_INVALID;
|
||||||
pReceiver->snapshot.lastApplyTerm = 0;
|
pReceiver->snapshot.lastApplyTerm = 0;
|
||||||
|
@ -474,19 +424,8 @@ bool snapshotReceiverIsStart(SSyncSnapshotReceiver *pReceiver) { return pReceive
|
||||||
// receive first snapshot data
|
// receive first snapshot data
|
||||||
// write first block data
|
// write first block data
|
||||||
static void snapshotReceiverDoStart(SSyncSnapshotReceiver *pReceiver, SyncSnapshotSend *pBeginMsg) {
|
static void snapshotReceiverDoStart(SSyncSnapshotReceiver *pReceiver, SyncSnapshotSend *pBeginMsg) {
|
||||||
// update state
|
|
||||||
pReceiver->term = pReceiver->pSyncNode->pRaftStore->currentTerm;
|
|
||||||
pReceiver->privateTerm = pBeginMsg->privateTerm;
|
|
||||||
pReceiver->ack = SYNC_SNAPSHOT_SEQ_BEGIN;
|
|
||||||
pReceiver->fromId = pBeginMsg->srcId;
|
|
||||||
pReceiver->start = true;
|
pReceiver->start = true;
|
||||||
|
pReceiver->ack = SYNC_SNAPSHOT_SEQ_BEGIN;
|
||||||
// update snapshot
|
|
||||||
pReceiver->snapshot.lastApplyIndex = pBeginMsg->lastIndex;
|
|
||||||
pReceiver->snapshot.lastApplyTerm = pBeginMsg->lastTerm;
|
|
||||||
pReceiver->snapshot.lastConfigIndex = pBeginMsg->lastConfigIndex;
|
|
||||||
pReceiver->snapshotParam.start = pBeginMsg->beginIndex;
|
|
||||||
pReceiver->snapshotParam.end = pBeginMsg->lastIndex;
|
|
||||||
|
|
||||||
// start writer
|
// start writer
|
||||||
ASSERT(pReceiver->pWriter == NULL);
|
ASSERT(pReceiver->pWriter == NULL);
|
||||||
|
@ -494,6 +433,19 @@ static void snapshotReceiverDoStart(SSyncSnapshotReceiver *pReceiver, SyncSnapsh
|
||||||
&(pReceiver->snapshotParam), &(pReceiver->pWriter));
|
&(pReceiver->snapshotParam), &(pReceiver->pWriter));
|
||||||
ASSERT(ret == 0);
|
ASSERT(ret == 0);
|
||||||
|
|
||||||
|
pReceiver->term = pReceiver->pSyncNode->pRaftStore->currentTerm;
|
||||||
|
pReceiver->snapshotParam.start = pBeginMsg->beginIndex;
|
||||||
|
pReceiver->snapshotParam.end = pBeginMsg->lastIndex;
|
||||||
|
|
||||||
|
pReceiver->fromId = pBeginMsg->srcId;
|
||||||
|
|
||||||
|
// update snapshot
|
||||||
|
pReceiver->snapshot.lastApplyIndex = pBeginMsg->lastIndex;
|
||||||
|
pReceiver->snapshot.lastApplyTerm = pBeginMsg->lastTerm;
|
||||||
|
pReceiver->snapshot.lastConfigIndex = pBeginMsg->lastConfigIndex;
|
||||||
|
|
||||||
|
pReceiver->startTime = pBeginMsg->startTime;
|
||||||
|
|
||||||
// event log
|
// event log
|
||||||
do {
|
do {
|
||||||
char *eventLog = snapshotReceiver2SimpleStr(pReceiver, "snapshot receiver start");
|
char *eventLog = snapshotReceiver2SimpleStr(pReceiver, "snapshot receiver start");
|
||||||
|
@ -523,22 +475,9 @@ void snapshotReceiverForceStop(SSyncSnapshotReceiver *pReceiver) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// if receiver receive msg from seq = SYNC_SNAPSHOT_SEQ_BEGIN, start receiver
|
// if receiver receive msg from seq = SYNC_SNAPSHOT_SEQ_BEGIN, start receiver
|
||||||
// if already start, force close, start again
|
|
||||||
int32_t snapshotReceiverStart(SSyncSnapshotReceiver *pReceiver, SyncSnapshotSend *pBeginMsg) {
|
int32_t snapshotReceiverStart(SSyncSnapshotReceiver *pReceiver, SyncSnapshotSend *pBeginMsg) {
|
||||||
if (!snapshotReceiverIsStart(pReceiver)) {
|
ASSERT(!snapshotReceiverIsStart(pReceiver));
|
||||||
// first start
|
snapshotReceiverDoStart(pReceiver, pBeginMsg);
|
||||||
snapshotReceiverDoStart(pReceiver, pBeginMsg);
|
|
||||||
|
|
||||||
} else {
|
|
||||||
// already start
|
|
||||||
sInfo("vgId:%d, snapshot recv, receiver already start", pReceiver->pSyncNode->vgId);
|
|
||||||
|
|
||||||
// force close, abandon incomplete data
|
|
||||||
snapshotReceiverForceStop(pReceiver);
|
|
||||||
|
|
||||||
// start again
|
|
||||||
snapshotReceiverDoStart(pReceiver, pBeginMsg);
|
|
||||||
}
|
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -698,8 +637,8 @@ cJSON *snapshotReceiver2Json(SSyncSnapshotReceiver *pReceiver) {
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pReceiver->term);
|
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pReceiver->term);
|
||||||
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
cJSON_AddStringToObject(pRoot, "term", u64buf);
|
||||||
|
|
||||||
snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pReceiver->privateTerm);
|
snprintf(u64buf, sizeof(u64buf), "%" PRId64, pReceiver->startTime);
|
||||||
cJSON_AddStringToObject(pRoot, "privateTerm", u64buf);
|
cJSON_AddStringToObject(pRoot, "startTime", u64buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
cJSON *pJson = cJSON_CreateObject();
|
cJSON *pJson = cJSON_CreateObject();
|
||||||
|
@ -724,38 +663,204 @@ char *snapshotReceiver2SimpleStr(SSyncSnapshotReceiver *pReceiver, char *event)
|
||||||
syncUtilU642Addr(fromId.addr, host, sizeof(host), &port);
|
syncUtilU642Addr(fromId.addr, host, sizeof(host), &port);
|
||||||
|
|
||||||
snprintf(s, len,
|
snprintf(s, len,
|
||||||
"%s {%p start:%d ack:%d term:%" PRIu64 " pterm:%" PRIu64 " from:%s:%d s-param:%" PRId64 " e-param:%" PRId64
|
"%s {%p start:%d ack:%d term:%" PRIu64 " start-time:%" PRId64 " from:%s:%d s-param:%" PRId64
|
||||||
" laindex:%" PRId64 " laterm:%" PRIu64
|
" e-param:%" PRId64 " laindex:%" PRId64 " laterm:%" PRIu64
|
||||||
" "
|
" "
|
||||||
"lcindex:%" PRId64 "}",
|
"lcindex:%" PRId64 "}",
|
||||||
event, pReceiver, pReceiver->start, pReceiver->ack, pReceiver->term, pReceiver->privateTerm, host, port,
|
event, pReceiver, pReceiver->start, pReceiver->ack, pReceiver->term, pReceiver->startTime, host, port,
|
||||||
pReceiver->snapshotParam.start, pReceiver->snapshotParam.end, pReceiver->snapshot.lastApplyIndex,
|
pReceiver->snapshotParam.start, pReceiver->snapshotParam.end, pReceiver->snapshot.lastApplyIndex,
|
||||||
pReceiver->snapshot.lastApplyTerm, pReceiver->snapshot.lastConfigIndex);
|
pReceiver->snapshot.lastApplyTerm, pReceiver->snapshot.lastConfigIndex);
|
||||||
|
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
SyncIndex syncNodeGetSnapBeginIndex(SSyncNode *ths) {
|
||||||
|
SyncIndex snapStart = SYNC_INDEX_INVALID;
|
||||||
|
|
||||||
|
if (syncNodeIsMnode(ths)) {
|
||||||
|
snapStart = SYNC_INDEX_BEGIN;
|
||||||
|
|
||||||
|
} else {
|
||||||
|
SSyncLogStoreData *pData = ths->pLogStore->data;
|
||||||
|
SWal *pWal = pData->pWal;
|
||||||
|
|
||||||
|
bool isEmpty = ths->pLogStore->syncLogIsEmpty(ths->pLogStore);
|
||||||
|
int64_t walCommitVer = walGetCommittedVer(pWal);
|
||||||
|
|
||||||
|
if (!isEmpty && ths->commitIndex != walCommitVer) {
|
||||||
|
char logBuf[128];
|
||||||
|
snprintf(logBuf, sizeof(logBuf), "commit not same, wal-commit:%" PRId64 ", commit:%" PRId64 ", ignore",
|
||||||
|
walCommitVer, ths->commitIndex);
|
||||||
|
syncNodeErrorLog(ths, logBuf);
|
||||||
|
|
||||||
|
snapStart = walCommitVer + 1;
|
||||||
|
} else {
|
||||||
|
snapStart = ths->commitIndex + 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return snapStart;
|
||||||
|
}
|
||||||
|
|
||||||
|
static int32_t syncNodeOnSnapshotPre(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
|
SSyncSnapshotReceiver *pReceiver = pSyncNode->pNewNodeReceiver;
|
||||||
|
|
||||||
|
if (snapshotReceiverIsStart(pReceiver)) {
|
||||||
|
// already start
|
||||||
|
|
||||||
|
if (pMsg->startTime > pReceiver->startTime) {
|
||||||
|
goto _START_RECEIVER;
|
||||||
|
|
||||||
|
} else if (pMsg->startTime == pReceiver->startTime) {
|
||||||
|
goto _SEND_REPLY;
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// ignore
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// start new
|
||||||
|
goto _START_RECEIVER;
|
||||||
|
}
|
||||||
|
|
||||||
|
_START_RECEIVER:
|
||||||
|
if (taosGetTimestampMs() - pMsg->startTime > SNAPSHOT_MAX_CLOCK_SKEW_MS) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "snapshot receiver time skew too much");
|
||||||
|
return -1;
|
||||||
|
} else {
|
||||||
|
// waiting for clock match
|
||||||
|
while (taosGetTimestampMs() > pMsg->startTime) {
|
||||||
|
taosMsleep(10);
|
||||||
|
}
|
||||||
|
|
||||||
|
snapshotReceiverStart(pReceiver, pMsg); // set start-time same with sender
|
||||||
|
}
|
||||||
|
|
||||||
|
_SEND_REPLY:
|
||||||
|
// build msg
|
||||||
|
; // make complier happy
|
||||||
|
SyncSnapshotRsp *pRspMsg = syncSnapshotRspBuild(pSyncNode->vgId);
|
||||||
|
pRspMsg->srcId = pSyncNode->myRaftId;
|
||||||
|
pRspMsg->destId = pMsg->srcId;
|
||||||
|
pRspMsg->term = pSyncNode->pRaftStore->currentTerm;
|
||||||
|
pRspMsg->lastIndex = pMsg->lastIndex;
|
||||||
|
pRspMsg->lastTerm = pMsg->lastTerm;
|
||||||
|
pRspMsg->startTime = pReceiver->startTime;
|
||||||
|
pRspMsg->ack = pMsg->seq; // receiver maybe already closed
|
||||||
|
pRspMsg->code = 0;
|
||||||
|
pRspMsg->snapBeginIndex = syncNodeGetSnapBeginIndex(pSyncNode);
|
||||||
|
|
||||||
|
// send msg
|
||||||
|
SRpcMsg rpcMsg;
|
||||||
|
syncSnapshotRsp2RpcMsg(pRspMsg, &rpcMsg);
|
||||||
|
syncNodeSendMsgById(&(pRspMsg->destId), pSyncNode, &rpcMsg);
|
||||||
|
syncSnapshotRspDestroy(pRspMsg);
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
static int32_t syncNodeOnSnapshotBegin(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
|
// condition 1
|
||||||
|
SSyncSnapshotReceiver *pReceiver = pSyncNode->pNewNodeReceiver;
|
||||||
|
|
||||||
|
if (snapshotReceiverIsStart(pReceiver)) {
|
||||||
|
if (pMsg->startTime > pReceiver->startTime) {
|
||||||
|
snapshotReceiverStop(pReceiver);
|
||||||
|
|
||||||
|
} else if (pMsg->startTime == pReceiver->startTime) {
|
||||||
|
return 0;
|
||||||
|
} else {
|
||||||
|
// ignore
|
||||||
|
syncNodeEventLog(pSyncNode, "msg ignore");
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
_START_RECEIVER:
|
||||||
|
if (taosGetTimestampMs() - pMsg->startTime > SNAPSHOT_MAX_CLOCK_SKEW_MS) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "snapshot receiver time skew too much");
|
||||||
|
return -1;
|
||||||
|
} else {
|
||||||
|
// waiting for clock match
|
||||||
|
while (taosGetTimestampMs() > pMsg->startTime) {
|
||||||
|
taosMsleep(10);
|
||||||
|
}
|
||||||
|
|
||||||
|
snapshotReceiverStart(pReceiver, pMsg);
|
||||||
|
|
||||||
|
// build msg
|
||||||
|
SyncSnapshotRsp *pRspMsg = syncSnapshotRspBuild(pSyncNode->vgId);
|
||||||
|
pRspMsg->srcId = pSyncNode->myRaftId;
|
||||||
|
pRspMsg->destId = pMsg->srcId;
|
||||||
|
pRspMsg->term = pSyncNode->pRaftStore->currentTerm;
|
||||||
|
pRspMsg->lastIndex = pMsg->lastIndex;
|
||||||
|
pRspMsg->lastTerm = pMsg->lastTerm;
|
||||||
|
pRspMsg->ack = pReceiver->ack; // receiver maybe already closed
|
||||||
|
pRspMsg->code = 0;
|
||||||
|
|
||||||
|
// send msg
|
||||||
|
SRpcMsg rpcMsg;
|
||||||
|
syncSnapshotRsp2RpcMsg(pRspMsg, &rpcMsg);
|
||||||
|
syncNodeSendMsgById(&(pRspMsg->destId), pSyncNode, &rpcMsg);
|
||||||
|
syncSnapshotRspDestroy(pRspMsg);
|
||||||
|
}
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
static int32_t syncNodeOnSnapshotTransfer(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) { return 0; }
|
||||||
|
|
||||||
|
static int32_t syncNodeOnSnapshotEnd(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) { return 0; }
|
||||||
|
|
||||||
// receiver on message
|
// receiver on message
|
||||||
//
|
//
|
||||||
// condition 1, recv SYNC_SNAPSHOT_SEQ_BEGIN, start receiver, update privateTerm
|
// condition 1, recv SYNC_SNAPSHOT_SEQ_PRE_SNAPSHOT
|
||||||
// condition 2, recv SYNC_SNAPSHOT_SEQ_END, finish receiver(apply snapshot data, update commit index, maybe reconfig)
|
// if receiver already start
|
||||||
// condition 3, recv SYNC_SNAPSHOT_SEQ_FORCE_CLOSE, force close
|
// if sender.start-time > receiver.start-time, restart receiver(reply snapshot start)
|
||||||
// condition 4, got data, update ack
|
// if sender.start-time = receiver.start-time, maybe duplicate msg
|
||||||
|
// if sender.start-time < receiver.start-time, ignore
|
||||||
|
// else
|
||||||
|
// waiting for clock match
|
||||||
|
// start receiver(reply snapshot start)
|
||||||
|
//
|
||||||
|
// condition 2, recv SYNC_SNAPSHOT_SEQ_BEGIN
|
||||||
|
// a. create writer with <begin, end>
|
||||||
|
//
|
||||||
|
// condition 3, recv SYNC_SNAPSHOT_SEQ_END, finish receiver(apply snapshot data, update commit index, maybe reconfig)
|
||||||
|
//
|
||||||
|
// condition 4, recv SYNC_SNAPSHOT_SEQ_FORCE_CLOSE, force close
|
||||||
|
//
|
||||||
|
// condition 5, got data, update ack
|
||||||
//
|
//
|
||||||
int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
// get receiver
|
// if already drop replica, do not process
|
||||||
SSyncSnapshotReceiver *pReceiver = pSyncNode->pNewNodeReceiver;
|
if (!syncNodeInRaftGroup(pSyncNode, &(pMsg->srcId))) {
|
||||||
bool needRsp = false;
|
syncLogRecvSyncSnapshotSend(pSyncNode, pMsg, "not in my config");
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (pMsg->term < pSyncNode->pRaftStore->currentTerm) {
|
||||||
|
syncLogRecvSyncSnapshotSend(pSyncNode, pMsg, "reject, small term");
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (pMsg->term > pSyncNode->pRaftStore->currentTerm) {
|
||||||
|
syncNodeStepDown(pSyncNode, pMsg->term);
|
||||||
|
}
|
||||||
|
syncNodeResetElectTimer(pSyncNode);
|
||||||
|
|
||||||
int32_t code = 0;
|
int32_t code = 0;
|
||||||
|
SSyncSnapshotReceiver *pReceiver = pSyncNode->pNewNodeReceiver;
|
||||||
|
|
||||||
// state, term, seq/ack
|
// state, term, seq/ack
|
||||||
if (pSyncNode->state == TAOS_SYNC_STATE_FOLLOWER) {
|
if (pSyncNode->state == TAOS_SYNC_STATE_FOLLOWER) {
|
||||||
if (pMsg->term == pSyncNode->pRaftStore->currentTerm) {
|
if (pMsg->term == pSyncNode->pRaftStore->currentTerm) {
|
||||||
if (pMsg->seq == SYNC_SNAPSHOT_SEQ_BEGIN) {
|
if (pMsg->seq == SYNC_SNAPSHOT_SEQ_PRE_SNAPSHOT) {
|
||||||
// condition 1
|
syncNodeOnSnapshotPre(pSyncNode, pMsg);
|
||||||
// begin, no data
|
|
||||||
snapshotReceiverStart(pReceiver, pMsg);
|
} else if (pMsg->seq == SYNC_SNAPSHOT_SEQ_BEGIN) {
|
||||||
needRsp = true;
|
syncNodeOnSnapshotBegin(pSyncNode, pMsg);
|
||||||
|
|
||||||
} else if (pMsg->seq == SYNC_SNAPSHOT_SEQ_END) {
|
} else if (pMsg->seq == SYNC_SNAPSHOT_SEQ_END) {
|
||||||
// condition 2
|
// condition 2
|
||||||
|
@ -764,7 +869,7 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
if (code == 0) {
|
if (code == 0) {
|
||||||
snapshotReceiverStop(pReceiver);
|
snapshotReceiverStop(pReceiver);
|
||||||
}
|
}
|
||||||
needRsp = true;
|
bool needRsp = true;
|
||||||
|
|
||||||
// maybe update lastconfig
|
// maybe update lastconfig
|
||||||
if (pMsg->lastConfigIndex >= SYNC_INDEX_BEGIN) {
|
if (pMsg->lastConfigIndex >= SYNC_INDEX_BEGIN) {
|
||||||
|
@ -782,7 +887,7 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
// condition 3
|
// condition 3
|
||||||
// force close
|
// force close
|
||||||
snapshotReceiverForceStop(pReceiver);
|
snapshotReceiverForceStop(pReceiver);
|
||||||
needRsp = false;
|
bool needRsp = false;
|
||||||
|
|
||||||
} else if (pMsg->seq > SYNC_SNAPSHOT_SEQ_BEGIN && pMsg->seq < SYNC_SNAPSHOT_SEQ_END) {
|
} else if (pMsg->seq > SYNC_SNAPSHOT_SEQ_BEGIN && pMsg->seq < SYNC_SNAPSHOT_SEQ_END) {
|
||||||
// condition 4
|
// condition 4
|
||||||
|
@ -790,7 +895,7 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
if (pMsg->seq == pReceiver->ack + 1) {
|
if (pMsg->seq == pReceiver->ack + 1) {
|
||||||
snapshotReceiverGotData(pReceiver, pMsg);
|
snapshotReceiverGotData(pReceiver, pMsg);
|
||||||
}
|
}
|
||||||
needRsp = true;
|
bool needRsp = true;
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
// error log
|
// error log
|
||||||
|
@ -805,26 +910,6 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
// send ack
|
|
||||||
if (needRsp) {
|
|
||||||
// build msg
|
|
||||||
SyncSnapshotRsp *pRspMsg = syncSnapshotRspBuild(pSyncNode->vgId);
|
|
||||||
pRspMsg->srcId = pSyncNode->myRaftId;
|
|
||||||
pRspMsg->destId = pMsg->srcId;
|
|
||||||
pRspMsg->term = pSyncNode->pRaftStore->currentTerm;
|
|
||||||
pRspMsg->lastIndex = pMsg->lastIndex;
|
|
||||||
pRspMsg->lastTerm = pMsg->lastTerm;
|
|
||||||
pRspMsg->ack = pReceiver->ack; // receiver maybe already closed
|
|
||||||
pRspMsg->code = 0;
|
|
||||||
pRspMsg->privateTerm = pReceiver->privateTerm; // receiver maybe already closed
|
|
||||||
|
|
||||||
// send msg
|
|
||||||
SRpcMsg rpcMsg;
|
|
||||||
syncSnapshotRsp2RpcMsg(pRspMsg, &rpcMsg);
|
|
||||||
syncNodeSendMsgById(&(pRspMsg->destId), pSyncNode, &rpcMsg);
|
|
||||||
syncSnapshotRspDestroy(pRspMsg);
|
|
||||||
}
|
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
// error log
|
// error log
|
||||||
do {
|
do {
|
||||||
|
@ -849,6 +934,52 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
int32_t syncNodeOnSnapshotReplyPre(SSyncNode *pSyncNode, SyncSnapshotRsp *pMsg) {
|
||||||
|
// get sender
|
||||||
|
SSyncSnapshotSender *pSender = syncNodeGetSnapshotSender(pSyncNode, &(pMsg->srcId));
|
||||||
|
ASSERT(pSender != NULL);
|
||||||
|
|
||||||
|
SSnapshot snapshot;
|
||||||
|
pSyncNode->pFsm->FpGetSnapshotInfo(pSyncNode->pFsm, &snapshot);
|
||||||
|
|
||||||
|
// prepare <begin, end>
|
||||||
|
pSender->snapshotParam.start = pMsg->snapBeginIndex;
|
||||||
|
pSender->snapshotParam.end = snapshot.lastApplyIndex;
|
||||||
|
|
||||||
|
if (pMsg->snapBeginIndex > snapshot.lastApplyIndex) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "snapshot last index too small");
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// start reader
|
||||||
|
int32_t code = pSyncNode->pFsm->FpSnapshotStartRead(pSyncNode->pFsm, &(pSender->snapshotParam), &(pSender->pReader));
|
||||||
|
if (code != 0) {
|
||||||
|
syncNodeErrorLog(pSyncNode, "create snapshot reader error");
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// build begin msg
|
||||||
|
SyncSnapshotSend *pSendMsg = syncSnapshotSendBuild(0, pSender->pSyncNode->vgId);
|
||||||
|
pSendMsg->srcId = pSender->pSyncNode->myRaftId;
|
||||||
|
pSendMsg->destId = (pSender->pSyncNode->replicasId)[pSender->replicaIndex];
|
||||||
|
pSendMsg->term = pSender->pSyncNode->pRaftStore->currentTerm;
|
||||||
|
pSendMsg->beginIndex = pSender->snapshotParam.start;
|
||||||
|
pSendMsg->lastIndex = pSender->snapshot.lastApplyIndex;
|
||||||
|
pSendMsg->lastTerm = pSender->snapshot.lastApplyTerm;
|
||||||
|
pSendMsg->lastConfigIndex = pSender->snapshot.lastConfigIndex;
|
||||||
|
pSendMsg->lastConfig = pSender->lastConfig;
|
||||||
|
pSendMsg->startTime = pSender->startTime;
|
||||||
|
pSendMsg->seq = SYNC_SNAPSHOT_SEQ_BEGIN;
|
||||||
|
|
||||||
|
// send msg
|
||||||
|
SRpcMsg rpcMsg;
|
||||||
|
syncSnapshotSend2RpcMsg(pSendMsg, &rpcMsg);
|
||||||
|
syncNodeSendMsgById(&(pSendMsg->destId), pSender->pSyncNode, &rpcMsg);
|
||||||
|
syncSnapshotSendDestroy(pSendMsg);
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
// sender on message
|
// sender on message
|
||||||
//
|
//
|
||||||
// condition 1 sender receives SYNC_SNAPSHOT_SEQ_END, close sender
|
// condition 1 sender receives SYNC_SNAPSHOT_SEQ_END, close sender
|
||||||
|
@ -857,8 +988,8 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
|
||||||
//
|
//
|
||||||
int32_t syncNodeOnSnapshotReply(SSyncNode *pSyncNode, SyncSnapshotRsp *pMsg) {
|
int32_t syncNodeOnSnapshotReply(SSyncNode *pSyncNode, SyncSnapshotRsp *pMsg) {
|
||||||
// if already drop replica, do not process
|
// if already drop replica, do not process
|
||||||
if (!syncNodeInRaftGroup(pSyncNode, &(pMsg->srcId)) && pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
|
if (!syncNodeInRaftGroup(pSyncNode, &(pMsg->srcId))) {
|
||||||
sError("vgId:%d, recv sync-snapshot-rsp, maybe replica already dropped", pSyncNode->vgId);
|
syncLogRecvSyncSnapshotRsp(pSyncNode, pMsg, "maybe replica already dropped");
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -866,17 +997,26 @@ int32_t syncNodeOnSnapshotReply(SSyncNode *pSyncNode, SyncSnapshotRsp *pMsg) {
|
||||||
SSyncSnapshotSender *pSender = syncNodeGetSnapshotSender(pSyncNode, &(pMsg->srcId));
|
SSyncSnapshotSender *pSender = syncNodeGetSnapshotSender(pSyncNode, &(pMsg->srcId));
|
||||||
ASSERT(pSender != NULL);
|
ASSERT(pSender != NULL);
|
||||||
|
|
||||||
|
if (pMsg->startTime != pSender->startTime) {
|
||||||
|
syncLogRecvSyncSnapshotRsp(pSyncNode, pMsg, "sender/receiver start time not match");
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
// state, term, seq/ack
|
// state, term, seq/ack
|
||||||
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
|
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
|
||||||
if (pMsg->term == pSyncNode->pRaftStore->currentTerm) {
|
if (pMsg->term == pSyncNode->pRaftStore->currentTerm) {
|
||||||
// condition 1
|
// prepare <begin, end>, send begin msg
|
||||||
|
if (pMsg->ack == SYNC_SNAPSHOT_SEQ_PRE_SNAPSHOT) {
|
||||||
|
syncNodeOnSnapshotReplyPre(pSyncNode, pMsg);
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
// receive ack is finish, close sender
|
// receive ack is finish, close sender
|
||||||
if (pMsg->ack == SYNC_SNAPSHOT_SEQ_END) {
|
if (pMsg->ack == SYNC_SNAPSHOT_SEQ_END) {
|
||||||
snapshotSenderStop(pSender, true);
|
snapshotSenderStop(pSender, true);
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
// condition 2
|
|
||||||
// send next msg
|
// send next msg
|
||||||
if (pMsg->ack == pSender->seq) {
|
if (pMsg->ack == pSender->seq) {
|
||||||
// update sender ack
|
// update sender ack
|
||||||
|
|
|
@ -50,7 +50,6 @@ SSyncSnapshotReceiver* createReceiver() {
|
||||||
pReceiver->ack = 20;
|
pReceiver->ack = 20;
|
||||||
pReceiver->pWriter = (void*)0x11;
|
pReceiver->pWriter = (void*)0x11;
|
||||||
pReceiver->term = 66;
|
pReceiver->term = 66;
|
||||||
pReceiver->privateTerm = 99;
|
|
||||||
|
|
||||||
return pReceiver;
|
return pReceiver;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,7 @@ SyncSnapshotRsp *createMsg() {
|
||||||
pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678);
|
pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678);
|
||||||
pMsg->destId.vgId = 100;
|
pMsg->destId.vgId = 100;
|
||||||
pMsg->term = 11;
|
pMsg->term = 11;
|
||||||
pMsg->privateTerm = 99;
|
pMsg->startTime = 99;
|
||||||
pMsg->lastIndex = 22;
|
pMsg->lastIndex = 22;
|
||||||
pMsg->lastTerm = 33;
|
pMsg->lastTerm = 33;
|
||||||
pMsg->ack = 44;
|
pMsg->ack = 44;
|
||||||
|
|
|
@ -21,7 +21,6 @@ SyncSnapshotSend *createMsg() {
|
||||||
pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678);
|
pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678);
|
||||||
pMsg->destId.vgId = 100;
|
pMsg->destId.vgId = 100;
|
||||||
pMsg->term = 11;
|
pMsg->term = 11;
|
||||||
pMsg->privateTerm = 99;
|
|
||||||
pMsg->lastIndex = 22;
|
pMsg->lastIndex = 22;
|
||||||
pMsg->lastTerm = 33;
|
pMsg->lastTerm = 33;
|
||||||
|
|
||||||
|
|
|
@ -55,7 +55,8 @@ SSyncSnapshotSender* createSender() {
|
||||||
pSender->snapshot.lastApplyTerm = 88;
|
pSender->snapshot.lastApplyTerm = 88;
|
||||||
pSender->sendingMS = 77;
|
pSender->sendingMS = 77;
|
||||||
pSender->term = 66;
|
pSender->term = 66;
|
||||||
pSender->privateTerm = 99;
|
|
||||||
|
//pSender->privateTerm = 99;
|
||||||
|
|
||||||
return pSender;
|
return pSender;
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue