提交 7808fdfc 编写于 作者: M Minghao Li

refactor(sync): add trace log

上级 c2b348be
...@@ -257,6 +257,22 @@ int32_t syncNodeLeaderTransfer(SSyncNode* pSyncNode); ...@@ -257,6 +257,22 @@ int32_t syncNodeLeaderTransfer(SSyncNode* pSyncNode);
int32_t syncNodeLeaderTransferTo(SSyncNode* pSyncNode, SNodeInfo newLeader); int32_t syncNodeLeaderTransferTo(SSyncNode* pSyncNode, SNodeInfo newLeader);
int32_t syncDoLeaderTransfer(SSyncNode* ths, SRpcMsg* pRpcMsg, SSyncRaftEntry* pEntry); int32_t syncDoLeaderTransfer(SSyncNode* ths, SRpcMsg* pRpcMsg, SSyncRaftEntry* pEntry);
// trace log
void syncLogSendRequestVote(SSyncNode* pSyncNode, const SyncRequestVote* pMsg, const char* s);
void syncLogRecvRequestVote(SSyncNode* pSyncNode, const SyncRequestVote* pMsg, const char* s);
void syncLogSendRequestVoteReply(SSyncNode* pSyncNode, const SyncRequestVoteReply* pMsg, const char* s);
void syncLogRecvRequestVoteReply(SSyncNode* pSyncNode, const SyncRequestVoteReply* pMsg, const char* s);
void syncLogSendAppendEntries(SSyncNode* pSyncNode, const SyncAppendEntries* pMsg, const char* s);
void syncLogRecvAppendEntries(SSyncNode* pSyncNode, const SyncAppendEntries* pMsg, const char* s);
void syncLogSendAppendEntriesBatch(SSyncNode* pSyncNode, const SyncAppendEntriesBatch* pMsg, const char* s);
void syncLogRecvAppendEntriesBatch(SSyncNode* pSyncNode, const SyncAppendEntriesBatch* pMsg, const char* s);
void syncLogSendAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntriesReply* pMsg, const char* s);
void syncLogRecvAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntriesReply* 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);
......
...@@ -42,36 +42,13 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p ...@@ -42,36 +42,13 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "maybe replica already dropped");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, maybe replica "
"already dropped",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
// drop stale response // drop stale response
if (pMsg->term < ths->pRaftStore->currentTerm) { if (pMsg->term < ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "drop stale response");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, drop stale response",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0);
return 0; return 0;
} }
...@@ -81,23 +58,15 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p ...@@ -81,23 +58,15 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p
// } // }
if (pMsg->term > ths->pRaftStore->currentTerm) { if (pMsg->term > ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "error term");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, error term",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
ASSERT(pMsg->term == ths->pRaftStore->currentTerm); ASSERT(pMsg->term == ths->pRaftStore->currentTerm);
SyncIndex beforeNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex beforeMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
if (pMsg->success) { if (pMsg->success) {
// nextIndex' = [nextIndex EXCEPT ![i][j] = m.mmatchIndex + 1] // nextIndex' = [nextIndex EXCEPT ![i][j] = m.mmatchIndex + 1]
syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), pMsg->matchIndex + 1); syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), pMsg->matchIndex + 1);
...@@ -120,20 +89,13 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p ...@@ -120,20 +89,13 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p
syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), nextIndex); syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), nextIndex);
} }
SyncIndex afterNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
do { do {
char host[64]; char logBuf[256];
uint16_t port; snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex,
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); beforeMatchIndex, afterNextIndex, afterMatchIndex);
char logBuf[256]; syncLogRecvAppendEntriesReply(ths, pMsg, logBuf);
SyncIndex nextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex matchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, after next:" PRId64
", "
"match:" PRId64 "",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex, nextIndex, matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0); } while (0);
return 0; return 0;
...@@ -179,58 +141,27 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie ...@@ -179,58 +141,27 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "maybe replica already dropped");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, maybe replica "
"already dropped",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
// drop stale response // drop stale response
if (pMsg->term < ths->pRaftStore->currentTerm) { if (pMsg->term < ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "drop stale response");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, drop stale response",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0);
return 0; return 0;
} }
// error term // error term
if (pMsg->term > ths->pRaftStore->currentTerm) { if (pMsg->term > ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "error term");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, error term",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
ASSERT(pMsg->term == ths->pRaftStore->currentTerm); ASSERT(pMsg->term == ths->pRaftStore->currentTerm);
SyncIndex beforeNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex beforeMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
if (pMsg->success) { if (pMsg->success) {
SyncIndex newNextIndex = pMsg->matchIndex + 1; SyncIndex newNextIndex = pMsg->matchIndex + 1;
SyncIndex newMatchIndex = pMsg->matchIndex; SyncIndex newMatchIndex = pMsg->matchIndex;
...@@ -343,20 +274,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie ...@@ -343,20 +274,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie
} while (0); } while (0);
} }
SyncIndex afterNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
do { do {
char host[64]; char logBuf[256];
uint16_t port; snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex,
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); beforeMatchIndex, afterNextIndex, afterMatchIndex);
char logBuf[256]; syncLogRecvAppendEntriesReply(ths, pMsg, logBuf);
SyncIndex nextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex matchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, after next:" PRId64
", "
"match:" PRId64 "",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex, nextIndex, matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0); } while (0);
return 0; return 0;
...@@ -367,36 +291,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries ...@@ -367,36 +291,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "maybe replica already dropped");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, maybe replica "
"already dropped",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
// drop stale response // drop stale response
if (pMsg->term < ths->pRaftStore->currentTerm) { if (pMsg->term < ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "drop stale response");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, drop stale response",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0);
return 0; return 0;
} }
...@@ -406,23 +307,15 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries ...@@ -406,23 +307,15 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries
// } // }
if (pMsg->term > ths->pRaftStore->currentTerm) { if (pMsg->term > ths->pRaftStore->currentTerm) {
do { syncLogRecvAppendEntriesReply(ths, pMsg, "error term");
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, error term",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex);
syncNodeErrorLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
ASSERT(pMsg->term == ths->pRaftStore->currentTerm); ASSERT(pMsg->term == ths->pRaftStore->currentTerm);
SyncIndex beforeNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex beforeMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
if (pMsg->success) { if (pMsg->success) {
// nextIndex' = [nextIndex EXCEPT ![i][j] = m.mmatchIndex + 1] // nextIndex' = [nextIndex EXCEPT ![i][j] = m.mmatchIndex + 1]
syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), pMsg->matchIndex + 1); syncIndexMgrSetIndex(ths->pNextIndex, &(pMsg->srcId), pMsg->matchIndex + 1);
...@@ -490,20 +383,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries ...@@ -490,20 +383,13 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries
} }
} }
SyncIndex afterNextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
do { do {
char host[64]; char logBuf[256];
uint16_t port; snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex,
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); beforeMatchIndex, afterNextIndex, afterMatchIndex);
char logBuf[256]; syncLogRecvAppendEntriesReply(ths, pMsg, logBuf);
SyncIndex nextIndex = syncIndexMgrGetIndex(ths->pNextIndex, &(pMsg->srcId));
SyncIndex matchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId));
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, after next:" PRId64
", "
"match:" PRId64 "",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex, nextIndex, matchIndex);
syncNodeEventLog(ths, logBuf);
} while (0); } while (0);
return 0; return 0;
......
...@@ -120,18 +120,7 @@ int32_t syncNodeElect(SSyncNode* pSyncNode) { ...@@ -120,18 +120,7 @@ int32_t syncNodeElect(SSyncNode* pSyncNode) {
int32_t syncNodeRequestVote(SSyncNode* pSyncNode, const SRaftId* destRaftId, const SyncRequestVote* pMsg) { int32_t syncNodeRequestVote(SSyncNode* pSyncNode, const SRaftId* destRaftId, const SyncRequestVote* pMsg) {
int32_t ret = 0; int32_t ret = 0;
syncLogSendRequestVote(pSyncNode, pMsg, "");
do {
char host[64];
uint16_t port;
syncUtilU642Addr(destRaftId->addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-request-vote to %s:%d {term:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64 "", host, port,
pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm);
syncNodeEventLog(pSyncNode, logBuf);
} while (0);
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
syncRequestVote2RpcMsg(pMsg, &rpcMsg); syncRequestVote2RpcMsg(pMsg, &rpcMsg);
......
...@@ -2925,4 +2925,126 @@ bool syncNodeCanChange(SSyncNode* pSyncNode) { ...@@ -2925,4 +2925,126 @@ bool syncNodeCanChange(SSyncNode* pSyncNode) {
} }
return true; return true;
} }
\ No newline at end of file
void syncLogSendRequestVote(SSyncNode* pSyncNode, const SyncRequestVote* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->destId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-request-vote to %s:%d {term:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64 "}, %s", host, port,
pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogRecvRequestVote(SSyncNode* pSyncNode, const SyncRequestVote* pMsg, const char* s) {
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:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64 "}, %s", host,
port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogSendRequestVoteReply(SSyncNode* pSyncNode, const SyncRequestVoteReply* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->destId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "send sync-request-vote-reply to %s:%d {term:%" PRIu64 ", grant:%d}, %s", host, port,
pMsg->term, pMsg->voteGranted, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogRecvRequestVoteReply(SSyncNode* pSyncNode, const SyncRequestVoteReply* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, %s", host,
port, pMsg->term, pMsg->voteGranted, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogSendAppendEntries(SSyncNode* pSyncNode, const SyncAppendEntries* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->destId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-append-entries to %s:%d, {term:%" PRIu64 ", pre-index:%" PRId64 ", pre-term:%" PRIu64
", pterm:%" PRIu64 ", commit:%" PRId64
", "
"datalen:%d}, %s",
host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm, pMsg->commitIndex,
pMsg->dataLen, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogRecvAppendEntries(SSyncNode* pSyncNode, const SyncAppendEntries* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries from %s:%d {term:" PRIu64 ", pre-index:" PRId64 ", pre-term:" PRIu64
", commit:" PRId64 ", pterm:" PRIu64
", "
"datalen:%d}, %s",
host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->commitIndex, pMsg->privateTerm,
pMsg->dataLen, s);
syncNodeErrorLog(pSyncNode, logBuf);
}
void syncLogSendAppendEntriesBatch(SSyncNode* pSyncNode, const SyncAppendEntriesBatch* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->destId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-append-entries-batch to %s:%d, {term:%" PRIu64 ", pre-index:%" PRId64 ", pre-term:%" PRIu64
", pterm:%" PRIu64 ", commit:%" PRId64 ", datalen:%d, count:%d}, %s",
pSyncNode->vgId, host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm,
pMsg->commitIndex, pMsg->dataLen, pMsg->dataCount, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogRecvAppendEntriesBatch(SSyncNode* pSyncNode, const SyncAppendEntriesBatch* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-batch from %s:%d, {term:%" PRIu64 ", pre-index:%" PRId64 ", pre-term:%" PRIu64
", pterm:%" PRIu64 ", commit:%" PRId64 ", datalen:%d, count:%d}, %s",
pSyncNode->vgId, host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm,
pMsg->commitIndex, pMsg->dataLen, pMsg->dataCount, s);
syncNodeErrorLog(pSyncNode, logBuf);
}
void syncLogSendAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntriesReply* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->destId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-append-entries-reply to %s:%d, {term:%" PRIu64 ", pterm:%" PRIu64 ", success:%d, match:%" PRId64
"}, %s",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex, s);
syncNodeEventLog(pSyncNode, logBuf);
}
void syncLogRecvAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntriesReply* pMsg, const char* s) {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-append-entries-reply from %s:%d {term:" PRIu64 ", pterm:" PRIu64 ", success:%d, match:" PRId64
"}, %s",
host, port, pMsg->term, pMsg->privateTerm, pMsg->success, pMsg->matchIndex, s);
syncNodeErrorLog(pSyncNode, logBuf);
}
...@@ -313,21 +313,7 @@ int32_t syncNodeReplicate(SSyncNode* pSyncNode) { ...@@ -313,21 +313,7 @@ int32_t syncNodeReplicate(SSyncNode* pSyncNode) {
int32_t syncNodeAppendEntries(SSyncNode* pSyncNode, const SRaftId* destRaftId, const SyncAppendEntries* pMsg) { int32_t syncNodeAppendEntries(SSyncNode* pSyncNode, const SRaftId* destRaftId, const SyncAppendEntries* pMsg) {
int32_t ret = 0; int32_t ret = 0;
syncLogSendAppendEntries(pSyncNode, pMsg, "");
do {
char host[64];
uint16_t port;
syncUtilU642Addr(destRaftId->addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-append-entries to %s:%d, {term:%" PRIu64 ", pre-index:%" PRId64 ", pre-term:%" PRIu64
", pterm:%" PRIu64 ", commit:%" PRId64
", "
"datalen:%d}",
host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm, pMsg->commitIndex,
pMsg->dataLen);
syncNodeEventLog(pSyncNode, logBuf);
} while (0);
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
syncAppendEntries2RpcMsg(pMsg, &rpcMsg); syncAppendEntries2RpcMsg(pMsg, &rpcMsg);
...@@ -337,18 +323,7 @@ int32_t syncNodeAppendEntries(SSyncNode* pSyncNode, const SRaftId* destRaftId, c ...@@ -337,18 +323,7 @@ int32_t syncNodeAppendEntries(SSyncNode* pSyncNode, const SRaftId* destRaftId, c
int32_t syncNodeAppendEntriesBatch(SSyncNode* pSyncNode, const SRaftId* destRaftId, int32_t syncNodeAppendEntriesBatch(SSyncNode* pSyncNode, const SRaftId* destRaftId,
const SyncAppendEntriesBatch* pMsg) { const SyncAppendEntriesBatch* pMsg) {
do { syncLogSendAppendEntriesBatch(pSyncNode, pMsg, "");
char host[64];
uint16_t port;
syncUtilU642Addr(destRaftId->addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"send sync-append-entries-batch to %s:%d, {term:%" PRIu64 ", pre-index:%" PRId64 ", pre-term:%" PRIu64
", pterm:%" PRIu64 ", commit:%" PRId64 ", datalen:%d, datacount:%d}",
pSyncNode->vgId, host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm,
pMsg->commitIndex, pMsg->dataLen, pMsg->dataCount);
syncNodeEventLog(pSyncNode, logBuf);
} while (0);
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
syncAppendEntriesBatch2RpcMsg(pMsg, &rpcMsg); syncAppendEntriesBatch2RpcMsg(pMsg, &rpcMsg);
......
...@@ -47,18 +47,7 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { ...@@ -47,18 +47,7 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) {
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
do { syncLogRecvRequestVote(ths, pMsg, "maybe replica already dropped");
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:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64
"}, maybe replica already dropped",
host, port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm);
syncNodeEventLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
...@@ -91,15 +80,10 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { ...@@ -91,15 +80,10 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) {
// trace log // trace log
do { do {
char logBuf[256]; char logBuf[32];
char host[64]; snprintf(logBuf, sizeof(logBuf), "grant:%d", pReply->voteGranted);
uint16_t port; syncLogRecvRequestVote(ths, pMsg, logBuf);
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); syncLogSendRequestVoteReply(ths, pReply, "");
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote from %s:%d, {term:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64
"}, reply-grant:%d",
host, port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm, pReply->voteGranted);
syncNodeEventLog(ths, logBuf);
} while (0); } while (0);
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
...@@ -212,18 +196,7 @@ int32_t syncNodeOnRequestVoteSnapshotCb(SSyncNode* ths, SyncRequestVote* pMsg) { ...@@ -212,18 +196,7 @@ int32_t syncNodeOnRequestVoteSnapshotCb(SSyncNode* ths, SyncRequestVote* pMsg) {
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
do { syncLogRecvRequestVote(ths, pMsg, "maybe replica already dropped");
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:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64
"}, maybe replica already dropped",
host, port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm);
syncNodeEventLog(ths, logBuf);
} while (0);
return -1; return -1;
} }
...@@ -254,15 +227,10 @@ int32_t syncNodeOnRequestVoteSnapshotCb(SSyncNode* ths, SyncRequestVote* pMsg) { ...@@ -254,15 +227,10 @@ int32_t syncNodeOnRequestVoteSnapshotCb(SSyncNode* ths, SyncRequestVote* pMsg) {
// trace log // trace log
do { do {
char logBuf[256]; char logBuf[32];
char host[64]; snprintf(logBuf, sizeof(logBuf), "grant:%d", pReply->voteGranted);
uint16_t port; syncLogRecvRequestVote(ths, pMsg, logBuf);
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port); syncLogSendRequestVoteReply(ths, pReply, "");
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote from %s:%d, {term:%" PRIu64 ", lindex:%" PRId64 ", lterm:%" PRIu64
"}, reply-grant:%d",
host, port, pMsg->term, pMsg->lastLogIndex, pMsg->lastLogTerm, pReply->voteGranted);
syncNodeEventLog(ths, logBuf);
} while (0); } while (0);
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
......
...@@ -40,40 +40,15 @@ ...@@ -40,40 +40,15 @@
int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) { int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) {
int32_t ret = 0; int32_t ret = 0;
// trace log
do {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d} ", host,
port, pMsg->term, pMsg->voteGranted);
syncNodeEventLog(ths, logBuf);
} while (0);
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "maybe replica already dropped");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, maybe replica dropped", host, port,
pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
// drop stale response // drop stale response
if (pMsg->term < ths->pRaftStore->currentTerm) { if (pMsg->term < ths->pRaftStore->currentTerm) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "drop stale response");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, drop stale response", host, port,
pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
...@@ -84,16 +59,11 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) ...@@ -84,16 +59,11 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg)
// } // }
if (pMsg->term > ths->pRaftStore->currentTerm) { if (pMsg->term > ths->pRaftStore->currentTerm) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "error term");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, error term",
host, port, pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
syncLogRecvRequestVoteReply(ths, pMsg, "");
ASSERT(pMsg->term == ths->pRaftStore->currentTerm); ASSERT(pMsg->term == ths->pRaftStore->currentTerm);
// This tallies votes even when the current state is not Candidate, // This tallies votes even when the current state is not Candidate,
...@@ -185,40 +155,15 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) ...@@ -185,40 +155,15 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg)
int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) { int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) {
int32_t ret = 0; int32_t ret = 0;
// trace log
do {
char host[64];
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d} ", host,
port, pMsg->term, pMsg->voteGranted);
syncNodeEventLog(ths, logBuf);
} while (0);
// if already drop replica, do not process // if already drop replica, do not process
if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) { if (!syncNodeInRaftGroup(ths, &(pMsg->srcId)) && !ths->pRaftCfg->isStandBy) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "maybe replica already dropped");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, maybe replica dropped", host, port,
pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
// drop stale response // drop stale response
if (pMsg->term < ths->pRaftStore->currentTerm) { if (pMsg->term < ths->pRaftStore->currentTerm) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "drop stale response");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf),
"recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, drop stale response", host, port,
pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
...@@ -229,16 +174,11 @@ int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteRepl ...@@ -229,16 +174,11 @@ int32_t syncNodeOnRequestVoteReplySnapshotCb(SSyncNode* ths, SyncRequestVoteRepl
// } // }
if (pMsg->term > ths->pRaftStore->currentTerm) { if (pMsg->term > ths->pRaftStore->currentTerm) {
char host[64]; syncLogRecvRequestVoteReply(ths, pMsg, "error term");
uint16_t port;
syncUtilU642Addr(pMsg->srcId.addr, host, sizeof(host), &port);
char logBuf[256];
snprintf(logBuf, sizeof(logBuf), "recv sync-request-vote-reply from %s:%d {term:%" PRIu64 ", grant:%d}, error term",
host, port, pMsg->term, pMsg->voteGranted);
syncNodeErrorLog(ths, logBuf);
return -1; return -1;
} }
syncLogRecvRequestVoteReply(ths, pMsg, "");
ASSERT(pMsg->term == ths->pRaftStore->currentTerm); ASSERT(pMsg->term == ths->pRaftStore->currentTerm);
// This tallies votes even when the current state is not Candidate, // This tallies votes even when the current state is not Candidate,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册