diff --git a/source/libs/sync/inc/syncInt.h b/source/libs/sync/inc/syncInt.h index 3085b6d8f4417cf88daa27192b05b53924b8630e..8a951ba38d08d4fc5f7a117780ef6a1e218924a4 100644 --- a/source/libs/sync/inc/syncInt.h +++ b/source/libs/sync/inc/syncInt.h @@ -268,8 +268,6 @@ int32_t syncNodeRestartHeartbeatTimer(SSyncNode* pSyncNode); // utils -------------- int32_t syncNodeSendMsgById(const SRaftId* destRaftId, SSyncNode* pSyncNode, SRpcMsg* pMsg); int32_t syncNodeSendMsgByInfo(const SNodeInfo* nodeInfo, SSyncNode* pSyncNode, SRpcMsg* pMsg); -cJSON* syncNode2Json(const SSyncNode* pSyncNode); -char* syncNode2Str(const SSyncNode* pSyncNode); char* syncNode2SimpleStr(const SSyncNode* pSyncNode); bool syncNodeInConfig(SSyncNode* pSyncNode, const SSyncCfg* config); void syncNodeDoConfigChange(SSyncNode* pSyncNode, SSyncCfg* newConfig, SyncIndex lastConfigChangeIndex); diff --git a/source/libs/sync/inc/syncRaftEntry.h b/source/libs/sync/inc/syncRaftEntry.h index bab1dcc661a90c0d9747d37699d64b747526495d..8b8ab41b53019528c4c9cbfe48f4f52441421fea 100644 --- a/source/libs/sync/inc/syncRaftEntry.h +++ b/source/libs/sync/inc/syncRaftEntry.h @@ -42,25 +42,14 @@ typedef struct SSyncRaftEntry { char data[]; // origin RpcMsg.pCont } SSyncRaftEntry; -SSyncRaftEntry* syncEntryBuild(uint32_t dataLen); -SSyncRaftEntry* syncEntryBuild2(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index); // step 4 -SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index); -SSyncRaftEntry* syncEntryBuild4(SRpcMsg* pOriginalMsg, SyncTerm term, SyncIndex index); +SSyncRaftEntry* syncEntryBuild(int32_t dataLen); +SSyncRaftEntry* syncEntryBuildFromClientRequest(const SyncClientRequest* pMsg, SyncTerm term, SyncIndex index); +SSyncRaftEntry* syncEntryBuildFromRpcMsg(const SRpcMsg* pMsg, SyncTerm term, SyncIndex index); +SSyncRaftEntry* syncEntryBuildFromAppendEntries(const SyncAppendEntries* pMsg); SSyncRaftEntry* syncEntryBuildNoop(SyncTerm term, SyncIndex index, int32_t vgId); void syncEntryDestory(SSyncRaftEntry* pEntry); -char* syncEntrySerialize(const SSyncRaftEntry* pEntry, uint32_t* len); // step 5 -SSyncRaftEntry* syncEntryDeserialize(const char* buf, uint32_t len); // step 6 -cJSON* syncEntry2Json(const SSyncRaftEntry* pEntry); -char* syncEntry2Str(const SSyncRaftEntry* pEntry); void syncEntry2OriginalRpc(const SSyncRaftEntry* pEntry, SRpcMsg* pRpcMsg); // step 7 -// for debug ---------------------- -void syncEntryPrint(const SSyncRaftEntry* pObj); -void syncEntryPrint2(char* s, const SSyncRaftEntry* pObj); -void syncEntryLog(const SSyncRaftEntry* pObj); -void syncEntryLog2(char* s, const SSyncRaftEntry* pObj); - -//----------------------------------- typedef struct SRaftEntryHashCache { SHashObj* pEntryHash; int32_t maxCount; @@ -78,14 +67,6 @@ int32_t raftCacheDelEntry(struct SRaftEntryHashCache* pCache, SyncI int32_t raftCacheGetAndDel(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry); int32_t raftCacheClear(struct SRaftEntryHashCache* pCache); -cJSON* raftCache2Json(SRaftEntryHashCache* pObj); -char* raftCache2Str(SRaftEntryHashCache* pObj); -void raftCachePrint(SRaftEntryHashCache* pObj); -void raftCachePrint2(char* s, SRaftEntryHashCache* pObj); -void raftCacheLog(SRaftEntryHashCache* pObj); -void raftCacheLog2(char* s, SRaftEntryHashCache* pObj); - -//----------------------------------- typedef struct SRaftEntryCache { SSkipList* pSkipList; int32_t maxCount; @@ -102,13 +83,6 @@ int32_t raftEntryCacheGetEntry(struct SRaftEntryCache* pCache, SyncInde int32_t raftEntryCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry); int32_t raftEntryCacheClear(struct SRaftEntryCache* pCache, int32_t count); -cJSON* raftEntryCache2Json(SRaftEntryCache* pObj); -char* raftEntryCache2Str(SRaftEntryCache* pObj); -void raftEntryCachePrint(SRaftEntryCache* pObj); -void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj); -void raftEntryCacheLog(SRaftEntryCache* pObj); -void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj); - #ifdef __cplusplus } #endif diff --git a/source/libs/sync/inc/syncRaftLog.h b/source/libs/sync/inc/syncRaftLog.h index ff59189a9d6a96566e3246a5ed8fa25ca819e440..c25d4ae34e5c1033ea727e097bc82a7d2f63270c 100644 --- a/source/libs/sync/inc/syncRaftLog.h +++ b/source/libs/sync/inc/syncRaftLog.h @@ -40,25 +40,19 @@ typedef struct SSyncLogStoreData { SSyncLogStore* logStoreCreate(SSyncNode* pSyncNode); void logStoreDestory(SSyncLogStore* pLogStore); -cJSON* logStore2Json(SSyncLogStore* pLogStore); -char* logStore2Str(SSyncLogStore* pLogStore); -cJSON* logStoreSimple2Json(SSyncLogStore* pLogStore); -char* logStoreSimple2Str(SSyncLogStore* pLogStore); SyncIndex logStoreFirstIndex(SSyncLogStore* pLogStore); SyncIndex logStoreWalCommitVer(SSyncLogStore* pLogStore); -// for debug -void logStorePrint(SSyncLogStore* pLogStore); -void logStorePrint2(char* s, SSyncLogStore* pLogStore); -void logStoreLog(SSyncLogStore* pLogStore); -void logStoreLog2(char* s, SSyncLogStore* pLogStore); - -void logStoreSimplePrint(SSyncLogStore* pLogStore); -void logStoreSimplePrint2(char* s, SSyncLogStore* pLogStore); -void logStoreSimpleLog(SSyncLogStore* pLogStore); -void logStoreSimpleLog2(char* s, SSyncLogStore* pLogStore); +SyncIndex raftLogWriteIndex(struct SSyncLogStore* pLogStore); +bool raftLogIsEmpty(struct SSyncLogStore* pLogStore); +SyncIndex raftLogBeginIndex(struct SSyncLogStore* pLogStore); +SyncIndex raftLogEndIndex(struct SSyncLogStore* pLogStore); +int32_t raftLogEntryCount(struct SSyncLogStore* pLogStore); +SyncIndex raftLogLastIndex(struct SSyncLogStore* pLogStore); +SyncTerm raftLogLastTerm(struct SSyncLogStore* pLogStore); +int32_t raftLogGetEntry(struct SSyncLogStore* pLogStore, SyncIndex index, SSyncRaftEntry** ppEntry); #ifdef __cplusplus } diff --git a/source/libs/sync/inc/syncTools.h b/source/libs/sync/inc/syncTools.h index 193579030f2c65b89bb74d21a4b9e87f38beb9dc..6a760ecd873b16fe944168b5238c286bcb8dc0f1 100644 --- a/source/libs/sync/inc/syncTools.h +++ b/source/libs/sync/inc/syncTools.h @@ -180,17 +180,13 @@ typedef struct SyncClientRequest { } SyncClientRequest; SyncClientRequest* syncClientRequestAlloc(uint32_t dataLen); -SyncClientRequest* syncClientRequestBuild(const SRpcMsg* pMsg, uint64_t seqNum, bool isWeak, int32_t vgId); // step 1 -void syncClientRequestDestroy(SyncClientRequest* pMsg); -void syncClientRequestSerialize(const SyncClientRequest* pMsg, char* buf, uint32_t bufLen); -void syncClientRequestDeserialize(const char* buf, uint32_t len, SyncClientRequest* pMsg); -char* syncClientRequestSerialize2(const SyncClientRequest* pMsg, uint32_t* len); -SyncClientRequest* syncClientRequestDeserialize2(const char* buf, uint32_t len); -void syncClientRequest2RpcMsg(const SyncClientRequest* pMsg, SRpcMsg* pRpcMsg); // step 2 -void syncClientRequestFromRpcMsg(const SRpcMsg* pRpcMsg, SyncClientRequest* pMsg); -SyncClientRequest* syncClientRequestFromRpcMsg2(const SRpcMsg* pRpcMsg); // step 3 -cJSON* syncClientRequest2Json(const SyncClientRequest* pMsg); -char* syncClientRequest2Str(const SyncClientRequest* pMsg); +int32_t syncClientRequestBuildFromRpcMsg(SRpcMsg* pClientRequestRpcMsg, const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, + bool isWeak, int32_t vgId); +int32_t syncClientRequestBuildFromNoopEntry(SRpcMsg* pClientRequestRpcMsg, const SSyncRaftEntry* pEntry, int32_t vgId); +void syncClientRequest2RpcMsg(const SyncClientRequest* pMsg, SRpcMsg* pRpcMsg); // step 2 +void syncClientRequestFromRpcMsg(const SRpcMsg* pRpcMsg, SyncClientRequest* pMsg); +cJSON* syncClientRequest2Json(const SyncClientRequest* pMsg); +char* syncClientRequest2Str(const SyncClientRequest* pMsg); // for debug ---------------------- void syncClientRequestPrint(const SyncClientRequest* pMsg); @@ -381,14 +377,6 @@ SyncAppendEntriesBatch* syncAppendEntriesBatchDeserialize2(const char* buf, uint void syncAppendEntriesBatch2RpcMsg(const SyncAppendEntriesBatch* pMsg, SRpcMsg* pRpcMsg); void syncAppendEntriesBatchFromRpcMsg(const SRpcMsg* pRpcMsg, SyncAppendEntriesBatch* pMsg); SyncAppendEntriesBatch* syncAppendEntriesBatchFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncAppendEntriesBatch2Json(const SyncAppendEntriesBatch* pMsg); -char* syncAppendEntriesBatch2Str(const SyncAppendEntriesBatch* pMsg); - -// for debug ---------------------- -void syncAppendEntriesBatchPrint(const SyncAppendEntriesBatch* pMsg); -void syncAppendEntriesBatchPrint2(char* s, const SyncAppendEntriesBatch* pMsg); -void syncAppendEntriesBatchLog(const SyncAppendEntriesBatch* pMsg); -void syncAppendEntriesBatchLog2(char* s, const SyncAppendEntriesBatch* pMsg); // --------------------------------------------- typedef struct SyncAppendEntriesReply { @@ -739,14 +727,14 @@ int32_t syncNodeOnSnapshotReply(SSyncNode* ths, SyncSnapshotRsp* pMsg); int32_t syncNodeOnHeartbeat(SSyncNode* ths, SyncHeartbeat* pMsg); int32_t syncNodeOnHeartbeatReply(SSyncNode* ths, SyncHeartbeatReply* pMsg); -int32_t syncNodeOnClientRequest(SSyncNode* ths, SyncClientRequest* pMsg, SyncIndex* pRetIndex); +int32_t syncNodeOnClientRequest(SSyncNode* ths, SRpcMsg* pMsg, SyncIndex* pRetIndex); int32_t syncNodeOnTimer(SSyncNode* ths, SyncTimeout* pMsg); int32_t syncNodeOnLocalCmd(SSyncNode* ths, SyncLocalCmd* pMsg); // ----------------------------------------- typedef int32_t (*FpOnPingCb)(SSyncNode* ths, SyncPing* pMsg); typedef int32_t (*FpOnPingReplyCb)(SSyncNode* ths, SyncPingReply* pMsg); -typedef int32_t (*FpOnClientRequestCb)(SSyncNode* ths, SyncClientRequest* pMsg, SyncIndex* pRetIndex); +typedef int32_t (*FpOnClientRequestCb)(SSyncNode* ths, SRpcMsg* pMsg, SyncIndex* pRetIndex); typedef int32_t (*FpOnRequestVoteCb)(SSyncNode* ths, SyncRequestVote* pMsg); typedef int32_t (*FpOnRequestVoteReplyCb)(SSyncNode* ths, SyncRequestVoteReply* pMsg); typedef int32_t (*FpOnAppendEntriesCb)(SSyncNode* ths, SyncAppendEntries* pMsg); diff --git a/source/libs/sync/src/syncAppendEntries.c b/source/libs/sync/src/syncAppendEntries.c index 7d6e3585110889e036fccbdea47bc0ea56501ab8..4f3f1c2c00681ecab8b24c515198f3c37166c52a 100644 --- a/source/libs/sync/src/syncAppendEntries.c +++ b/source/libs/sync/src/syncAppendEntries.c @@ -179,7 +179,7 @@ int32_t syncNodeOnAppendEntries(SSyncNode* ths, SyncAppendEntries* pMsg) { pReply->success = true; bool hasAppendEntries = pMsg->dataLen > 0; if (hasAppendEntries) { - SSyncRaftEntry* pAppendEntry = syncEntryDeserialize(pMsg->data, pMsg->dataLen); + SSyncRaftEntry* pAppendEntry = syncEntryBuildFromAppendEntries(pMsg); ASSERT(pAppendEntry != NULL); SyncIndex appendIndex = pMsg->prevLogIndex + 1; diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index 14823be0982be118d51163d52e5ec961fcdb7733..67d5c5afdd4a254f6c593b003c33737bf2d3f017 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -154,9 +154,7 @@ int32_t syncProcessMsg(int64_t rid, SRpcMsg* pMsg) { code = syncNodeOnPingReply(pSyncNode, pSyncMsg); syncPingReplyDestroy(pSyncMsg); } else if (pMsg->msgType == TDMT_SYNC_CLIENT_REQUEST) { - SyncClientRequest* pSyncMsg = syncClientRequestFromRpcMsg2(pMsg); - code = syncNodeOnClientRequest(pSyncNode, pSyncMsg, NULL); - syncClientRequestDestroy(pSyncMsg); + code = syncNodeOnClientRequest(pSyncNode, pMsg, NULL); } else if (pMsg->msgType == TDMT_SYNC_REQUEST_VOTE) { SyncRequestVote* pSyncMsg = syncRequestVoteFromRpcMsg2(pMsg); code = syncNodeOnRequestVote(pSyncNode, pSyncMsg); @@ -225,7 +223,7 @@ int32_t syncBeginSnapshot(int64_t rid, int64_t lastApplyIndex) { sError("sync begin snapshot error"); return -1; } - + int32_t code = 0; if (syncNodeIsMnode(pSyncNode)) { @@ -601,44 +599,42 @@ int32_t syncNodePropose(SSyncNode* pSyncNode, SRpcMsg* pMsg, bool isWeak) { return -1; } - int32_t ret = 0; - SyncClientRequest* pSyncMsg; - // optimized one replica if (syncNodeIsOptimizedOneReplica(pSyncNode, pMsg)) { - pSyncMsg = syncClientRequestBuild(pMsg, 0, isWeak, pSyncNode->vgId); - SyncIndex retIndex; - int32_t code = syncNodeOnClientRequest(pSyncNode, pSyncMsg, &retIndex); + int32_t code = syncNodeOnClientRequest(pSyncNode, pMsg, &retIndex); if (code == 0) { pMsg->info.conn.applyIndex = retIndex; pMsg->info.conn.applyTerm = pSyncNode->pRaftStore->currentTerm; - ret = 1; - sTrace("vgId:%d, sync optimize index:%" PRId64 ", type:%s", pSyncNode->vgId, retIndex, TMSG_INFO(pMsg->msgType)); + sTrace("vgId:%d, propose optimized msg, index:%" PRId64 " type:%s", pSyncNode->vgId, retIndex, + TMSG_INFO(pMsg->msgType)); + return 1; } else { - ret = -1; terrno = TSDB_CODE_SYN_INTERNAL_ERROR; - sError("vgId:%d, failed to sync optimize index:%" PRId64 ", type:%s", pSyncNode->vgId, retIndex, + sError("vgId:%d, failed to propose optimized msg, index:%" PRId64 " type:%s", pSyncNode->vgId, retIndex, TMSG_INFO(pMsg->msgType)); + return -1; } } else { SRespStub stub = {.createTime = taosGetTimestampMs(), .rpcMsg = *pMsg}; uint64_t seqNum = syncRespMgrAdd(pSyncNode->pSyncRespMgr, &stub); + SRpcMsg rpcMsg = {0}; + int32_t code = syncClientRequestBuildFromRpcMsg(&rpcMsg, pMsg, seqNum, isWeak, pSyncNode->vgId); + if (code != 0) { + sError("vgId:%d, failed to propose msg while serialize since %s", pSyncNode->vgId, terrstr()); + (void)syncRespMgrDel(pSyncNode->pSyncRespMgr, seqNum); + return -1; + } - pSyncMsg = syncClientRequestBuild(pMsg, seqNum, isWeak, pSyncNode->vgId); - SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pSyncMsg, &rpcMsg); - - sNTrace(pSyncNode, "propose message, type:%s", TMSG_INFO(pMsg->msgType)); - ret = (*pSyncNode->syncEqMsg)(pSyncNode->msgcb, &rpcMsg); - if (ret != 0) { - sError("vgId:%d, failed to enqueue msg since %s", pSyncNode->vgId, terrstr()); - syncRespMgrDel(pSyncNode->pSyncRespMgr, seqNum); + sNTrace(pSyncNode, "propose msg, type:%s", TMSG_INFO(pMsg->msgType)); + code = (*pSyncNode->syncEqMsg)(pSyncNode->msgcb, &rpcMsg); + if (code != 0) { + sError("vgId:%d, failed to propose msg while enqueue since %s", pSyncNode->vgId, terrstr()); + (void)syncRespMgrDel(pSyncNode->pSyncRespMgr, seqNum); } - } - syncClientRequestDestroy(pSyncMsg); - return ret; + return code; + } } static int32_t syncHbTimerInit(SSyncNode* pSyncNode, SSyncTimer* pSyncTimer, SRaftId destId) { @@ -1084,53 +1080,6 @@ void syncNodeClose(SSyncNode* pSyncNode) { ESyncStrategy syncNodeStrategy(SSyncNode* pSyncNode) { return pSyncNode->pRaftCfg->snapshotStrategy; } -// ping -------------- -int32_t syncNodePing(SSyncNode* pSyncNode, const SRaftId* destRaftId, SyncPing* pMsg) { - syncPingLog2((char*)"==syncNodePing==", pMsg); - int32_t ret = 0; - - SRpcMsg rpcMsg; - syncPing2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgLog2((char*)"==syncNodePing==", &rpcMsg); - - ret = syncNodeSendMsgById(destRaftId, pSyncNode, &rpcMsg); - return ret; -} - -int32_t syncNodePingSelf(SSyncNode* pSyncNode) { - int32_t ret = 0; - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &pSyncNode->myRaftId, pSyncNode->vgId); - ret = syncNodePing(pSyncNode, &pMsg->destId, pMsg); - ASSERT(ret == 0); - - syncPingDestroy(pMsg); - return ret; -} - -int32_t syncNodePingPeers(SSyncNode* pSyncNode) { - int32_t ret = 0; - for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { - SRaftId* destId = &(pSyncNode->peersId[i]); - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, destId, pSyncNode->vgId); - ret = syncNodePing(pSyncNode, destId, pMsg); - ASSERT(ret == 0); - syncPingDestroy(pMsg); - } - return ret; -} - -int32_t syncNodePingAll(SSyncNode* pSyncNode) { - int32_t ret = 0; - for (int32_t i = 0; i < pSyncNode->pRaftCfg->cfg.replicaNum; ++i) { - SRaftId* destId = &(pSyncNode->replicasId[i]); - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, destId, pSyncNode->vgId); - ret = syncNodePing(pSyncNode, destId, pMsg); - ASSERT(ret == 0); - syncPingDestroy(pMsg); - } - return ret; -} - // timer control -------------- int32_t syncNodeStartPingTimer(SSyncNode* pSyncNode) { int32_t ret = 0; @@ -1293,196 +1242,6 @@ int32_t syncNodeSendMsgByInfo(const SNodeInfo* nodeInfo, SSyncNode* pSyncNode, S return 0; } -cJSON* syncNode2Json(const SSyncNode* pSyncNode) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pSyncNode != NULL) { - // init by SSyncInfo - cJSON_AddNumberToObject(pRoot, "vgId", pSyncNode->vgId); - cJSON_AddItemToObject(pRoot, "SRaftCfg", raftCfg2Json(pSyncNode->pRaftCfg)); - cJSON_AddStringToObject(pRoot, "path", pSyncNode->path); - cJSON_AddStringToObject(pRoot, "raftStorePath", pSyncNode->raftStorePath); - cJSON_AddStringToObject(pRoot, "configPath", pSyncNode->configPath); - - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pWal); - cJSON_AddStringToObject(pRoot, "pWal", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->msgcb); - cJSON_AddStringToObject(pRoot, "rpcClient", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->syncSendMSg); - cJSON_AddStringToObject(pRoot, "syncSendMSg", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->msgcb); - cJSON_AddStringToObject(pRoot, "queue", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->syncEqMsg); - cJSON_AddStringToObject(pRoot, "syncEqMsg", u64buf); - - // init internal - cJSON* pMe = syncUtilNodeInfo2Json(&pSyncNode->myNodeInfo); - cJSON_AddItemToObject(pRoot, "myNodeInfo", pMe); - cJSON* pRaftId = syncUtilRaftId2Json(&pSyncNode->myRaftId); - cJSON_AddItemToObject(pRoot, "myRaftId", pRaftId); - - cJSON_AddNumberToObject(pRoot, "peersNum", pSyncNode->peersNum); - cJSON* pPeers = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "peersNodeInfo", pPeers); - for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { - cJSON_AddItemToArray(pPeers, syncUtilNodeInfo2Json(&pSyncNode->peersNodeInfo[i])); - } - cJSON* pPeersId = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "peersId", pPeersId); - for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { - cJSON_AddItemToArray(pPeersId, syncUtilRaftId2Json(&pSyncNode->peersId[i])); - } - - cJSON_AddNumberToObject(pRoot, "replicaNum", pSyncNode->replicaNum); - cJSON* pReplicasId = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "replicasId", pReplicasId); - for (int32_t i = 0; i < pSyncNode->replicaNum; ++i) { - cJSON_AddItemToArray(pReplicasId, syncUtilRaftId2Json(&pSyncNode->replicasId[i])); - } - - // raft algorithm - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pFsm); - cJSON_AddStringToObject(pRoot, "pFsm", u64buf); - cJSON_AddNumberToObject(pRoot, "quorum", pSyncNode->quorum); - cJSON* pLaderCache = syncUtilRaftId2Json(&pSyncNode->leaderCache); - cJSON_AddItemToObject(pRoot, "leaderCache", pLaderCache); - - // life cycle - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pSyncNode->rid); - cJSON_AddStringToObject(pRoot, "rid", u64buf); - - // tla+ server vars - cJSON_AddNumberToObject(pRoot, "state", pSyncNode->state); - cJSON_AddStringToObject(pRoot, "state_str", syncStr(pSyncNode->state)); - cJSON_AddItemToObject(pRoot, "pRaftStore", raftStore2Json(pSyncNode->pRaftStore)); - - // tla+ candidate vars - cJSON_AddItemToObject(pRoot, "pVotesGranted", voteGranted2Json(pSyncNode->pVotesGranted)); - cJSON_AddItemToObject(pRoot, "pVotesRespond", votesRespond2Json(pSyncNode->pVotesRespond)); - - // tla+ leader vars - cJSON_AddItemToObject(pRoot, "pNextIndex", syncIndexMgr2Json(pSyncNode->pNextIndex)); - cJSON_AddItemToObject(pRoot, "pMatchIndex", syncIndexMgr2Json(pSyncNode->pMatchIndex)); - - // tla+ log vars - cJSON_AddItemToObject(pRoot, "pLogStore", logStore2Json(pSyncNode->pLogStore)); - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pSyncNode->commitIndex); - cJSON_AddStringToObject(pRoot, "commitIndex", u64buf); - - // timer ms init - cJSON_AddNumberToObject(pRoot, "pingBaseLine", pSyncNode->pingBaseLine); - cJSON_AddNumberToObject(pRoot, "electBaseLine", pSyncNode->electBaseLine); - cJSON_AddNumberToObject(pRoot, "hbBaseLine", pSyncNode->hbBaseLine); - - // ping timer - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pPingTimer); - cJSON_AddStringToObject(pRoot, "pPingTimer", u64buf); - cJSON_AddNumberToObject(pRoot, "pingTimerMS", pSyncNode->pingTimerMS); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerLogicClock); - cJSON_AddStringToObject(pRoot, "pingTimerLogicClock", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerLogicClockUser); - cJSON_AddStringToObject(pRoot, "pingTimerLogicClockUser", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpPingTimerCB); - cJSON_AddStringToObject(pRoot, "FpPingTimerCB", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerCounter); - cJSON_AddStringToObject(pRoot, "pingTimerCounter", u64buf); - - // elect timer - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pElectTimer); - cJSON_AddStringToObject(pRoot, "pElectTimer", u64buf); - cJSON_AddNumberToObject(pRoot, "electTimerMS", pSyncNode->electTimerMS); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->electTimerLogicClock); - cJSON_AddStringToObject(pRoot, "electTimerLogicClock", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpElectTimerCB); - cJSON_AddStringToObject(pRoot, "FpElectTimerCB", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->electTimerCounter); - cJSON_AddStringToObject(pRoot, "electTimerCounter", u64buf); - - // heartbeat timer - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pHeartbeatTimer); - cJSON_AddStringToObject(pRoot, "pHeartbeatTimer", u64buf); - cJSON_AddNumberToObject(pRoot, "heartbeatTimerMS", pSyncNode->heartbeatTimerMS); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerLogicClock); - cJSON_AddStringToObject(pRoot, "heartbeatTimerLogicClock", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerLogicClockUser); - cJSON_AddStringToObject(pRoot, "heartbeatTimerLogicClockUser", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpHeartbeatTimerCB); - cJSON_AddStringToObject(pRoot, "FpHeartbeatTimerCB", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerCounter); - cJSON_AddStringToObject(pRoot, "heartbeatTimerCounter", u64buf); - - // callback - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnPing); - cJSON_AddStringToObject(pRoot, "FpOnPing", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnPingReply); - cJSON_AddStringToObject(pRoot, "FpOnPingReply", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnRequestVote); - cJSON_AddStringToObject(pRoot, "FpOnRequestVote", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnRequestVoteReply); - cJSON_AddStringToObject(pRoot, "FpOnRequestVoteReply", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnAppendEntries); - cJSON_AddStringToObject(pRoot, "FpOnAppendEntries", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnAppendEntriesReply); - cJSON_AddStringToObject(pRoot, "FpOnAppendEntriesReply", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnTimeout); - cJSON_AddStringToObject(pRoot, "FpOnTimeout", u64buf); - - // restoreFinish - cJSON_AddNumberToObject(pRoot, "restoreFinish", pSyncNode->restoreFinish); - - // snapshot senders - cJSON* pSenders = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "senders", pSenders); - for (int32_t i = 0; i < TSDB_MAX_REPLICA; ++i) { - cJSON_AddItemToArray(pSenders, snapshotSender2Json((pSyncNode->senders)[i])); - } - - // snapshot receivers - cJSON* pReceivers = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "receiver", snapshotReceiver2Json(pSyncNode->pNewNodeReceiver)); - - // changing - cJSON_AddNumberToObject(pRoot, "changing", pSyncNode->changing); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SSyncNode", pRoot); - return pJson; -} - -char* syncNode2Str(const SSyncNode* pSyncNode) { - cJSON* pJson = syncNode2Json(pSyncNode); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -inline char* syncNode2SimpleStr(const SSyncNode* pSyncNode) { - int32_t len = 256; - char* s = (char*)taosMemoryMalloc(len); - - SSnapshot snapshot = {.data = NULL, .lastApplyIndex = -1, .lastApplyTerm = 0}; - if (pSyncNode->pFsm->FpGetSnapshotInfo != NULL) { - pSyncNode->pFsm->FpGetSnapshotInfo(pSyncNode->pFsm, &snapshot); - } - SyncIndex logLastIndex = pSyncNode->pLogStore->syncLogLastIndex(pSyncNode->pLogStore); - SyncIndex logBeginIndex = pSyncNode->pLogStore->syncLogBeginIndex(pSyncNode->pLogStore); - - snprintf(s, len, - "vgId:%d, sync %s, tm:%" PRIu64 ", cmt:%" PRId64 ", fst:%" PRId64 ", lst:%" PRId64 ", snap:%" PRId64 - ", sby:%d, " - "r-num:%d, " - "lcfg:%" PRId64 ", chging:%d, rsto:%d", - pSyncNode->vgId, syncStr(pSyncNode->state), pSyncNode->pRaftStore->currentTerm, pSyncNode->commitIndex, - logBeginIndex, logLastIndex, snapshot.lastApplyIndex, pSyncNode->pRaftCfg->isStandBy, pSyncNode->replicaNum, - pSyncNode->pRaftCfg->lastConfigIndex, pSyncNode->changing, pSyncNode->restoreFinish); - - return s; -} - inline bool syncNodeInConfig(SSyncNode* pSyncNode, const SSyncCfg* config) { bool b1 = false; bool b2 = false; @@ -2056,8 +1815,6 @@ int32_t syncNodeGetPreIndexTerm(SSyncNode* pSyncNode, SyncIndex index, SyncIndex return 0; } -// ------ local funciton --------- -// enqueue message ---- static void syncNodeEqPingTimer(void* param, void* tmrId) { SSyncNode* pSyncNode = (SSyncNode*)param; if (atomic_load_64(&pSyncNode->pingTimerLogicClockUser) <= atomic_load_64(&pSyncNode->pingTimerLogicClock)) { @@ -2065,7 +1822,7 @@ static void syncNodeEqPingTimer(void* param, void* tmrId) { pSyncNode->pingTimerMS, pSyncNode->vgId, pSyncNode); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pSyncMsg, &rpcMsg); - syncRpcMsgLog2((char*)"==syncNodeEqPingTimer==", &rpcMsg); + sNTrace(pSyncNode, "enqueue ping timer"); if (pSyncNode->syncEqMsg != NULL) { int32_t code = pSyncNode->syncEqMsg(pSyncNode->msgcb, &rpcMsg); if (code != 0) { @@ -2141,7 +1898,7 @@ static void syncNodeEqHeartbeatTimer(void* param, void* tmrId) { pSyncNode->heartbeatTimerMS, pSyncNode->vgId, pSyncNode); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pSyncMsg, &rpcMsg); - syncRpcMsgLog2((char*)"==syncNodeEqHeartbeatTimer==", &rpcMsg); + sNTrace(pSyncNode, "enqueue heartbeat timer"); if (pSyncNode->syncEqMsg != NULL) { int32_t code = pSyncNode->syncEqMsg(pSyncNode->msgcb, &rpcMsg); if (code != 0) { @@ -2230,34 +1987,28 @@ static void syncNodeEqPeerHeartbeatTimer(void* param, void* tmrId) { } } -static int32_t syncNodeEqNoop(SSyncNode* ths) { - int32_t ret = 0; - ASSERT(ths->state == TAOS_SYNC_STATE_LEADER); - - SyncIndex index = ths->pLogStore->syncLogWriteIndex(ths->pLogStore); - SyncTerm term = ths->pRaftStore->currentTerm; - SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index, ths->vgId); - ASSERT(pEntry != NULL); +static int32_t syncNodeEqNoop(SSyncNode* pNode) { + if (pNode->state == TAOS_SYNC_STATE_LEADER) { + terrno = TSDB_CODE_SYN_NOT_LEADER; + return -1; + } - uint32_t entryLen; - char* serialized = syncEntrySerialize(pEntry, &entryLen); - SyncClientRequest* pSyncMsg = syncClientRequestAlloc(entryLen); - ASSERT(pSyncMsg->dataLen == entryLen); - memcpy(pSyncMsg->data, serialized, entryLen); + SyncIndex index = pNode->pLogStore->syncLogWriteIndex(pNode->pLogStore); + SyncTerm term = pNode->pRaftStore->currentTerm; + SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index, pNode->vgId); + if (pEntry == NULL) return -1; SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pSyncMsg, &rpcMsg); - if (ths->syncEqMsg != NULL) { - ths->syncEqMsg(ths->msgcb, &rpcMsg); - } else { - sTrace("syncNodeEqNoop pSyncNode->syncEqMsg is NULL"); - } - + int32_t code = syncClientRequestBuildFromNoopEntry(&rpcMsg, pEntry, pNode->vgId); syncEntryDestory(pEntry); - taosMemoryFree(serialized); - syncClientRequestDestroy(pSyncMsg); - return ret; + sNTrace(pNode, "propose msg, type:noop"); + code = (*pNode->syncEqMsg)(pNode->msgcb, &rpcMsg); + if (code != 0) { + sNError(pNode, "failed to propose noop msg while enqueue since %s", terrstr()); + } + + return code; } static void deleteCacheEntry(const void* key, size_t keyLen, void* value) { taosMemoryFree(value); } @@ -2438,7 +2189,7 @@ int32_t syncNodeOnLocalCmd(SSyncNode* ths, SyncLocalCmd* pMsg) { // leaderVars, commitIndex>> // -int32_t syncNodeOnClientRequest(SSyncNode* ths, SyncClientRequest* pMsg, SyncIndex* pRetIndex) { +int32_t syncNodeOnClientRequest(SSyncNode* ths, SRpcMsg* pMsg, SyncIndex* pRetIndex) { sNTrace(ths, "on client request"); int32_t ret = 0; @@ -2446,8 +2197,13 @@ int32_t syncNodeOnClientRequest(SSyncNode* ths, SyncClientRequest* pMsg, SyncInd SyncIndex index = ths->pLogStore->syncLogWriteIndex(ths->pLogStore); SyncTerm term = ths->pRaftStore->currentTerm; - SSyncRaftEntry* pEntry = syncEntryBuild2(pMsg, term, index); - ASSERT(pEntry != NULL); + SSyncRaftEntry* pEntry; + + if (pMsg->msgType == TDMT_SYNC_CLIENT_REQUEST) { + pEntry = syncEntryBuildFromClientRequest(pMsg->pCont, term, index); + } else { + pEntry = syncEntryBuildFromRpcMsg(pMsg, term, index); + } LRUHandle* h = NULL; syncCacheEntry(ths->pLogStore, pEntry, &h); @@ -2646,7 +2402,7 @@ int32_t syncNodeDoCommit(SSyncNode* ths, SyncIndex beginIndex, SyncIndex endInde } } - SRpcMsg rpcMsg; + SRpcMsg rpcMsg = {0}; syncEntry2OriginalRpc(pEntry, &rpcMsg); // user commit diff --git a/source/libs/sync/src/syncMessage.c b/source/libs/sync/src/syncMessage.c index 64536935762819d0d67296e5e09c0cd27bd58f05..3fcb563f3bced428398525c189fc2ead28b57dec 100644 --- a/source/libs/sync/src/syncMessage.c +++ b/source/libs/sync/src/syncMessage.c @@ -19,146 +19,6 @@ #include "syncUtil.h" #include "tcoding.h" -// --------------------------------------------- -cJSON* syncRpcMsg2Json(SRpcMsg* pRpcMsg) { - cJSON* pRoot; - - // in compiler optimization, switch case = if else constants - if (pRpcMsg->msgType == TDMT_SYNC_TIMEOUT) { - SyncTimeout* pSyncMsg = syncTimeoutDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncTimeout2Json(pSyncMsg); - syncTimeoutDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_PING) { - SyncPing* pSyncMsg = syncPingDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncPing2Json(pSyncMsg); - syncPingDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_PING_REPLY) { - SyncPingReply* pSyncMsg = syncPingReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncPingReply2Json(pSyncMsg); - syncPingReplyDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_CLIENT_REQUEST) { - SyncClientRequest* pSyncMsg = syncClientRequestDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncClientRequest2Json(pSyncMsg); - syncClientRequestDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_CLIENT_REQUEST_REPLY) { - pRoot = syncRpcUnknownMsg2Json(); - - } else if (pRpcMsg->msgType == TDMT_SYNC_REQUEST_VOTE) { - SyncRequestVote* pSyncMsg = syncRequestVoteDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncRequestVote2Json(pSyncMsg); - syncRequestVoteDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_REQUEST_VOTE_REPLY) { - SyncRequestVoteReply* pSyncMsg = syncRequestVoteReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncRequestVoteReply2Json(pSyncMsg); - syncRequestVoteReplyDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_APPEND_ENTRIES) { - SyncAppendEntries* pSyncMsg = syncAppendEntriesDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncAppendEntries2Json(pSyncMsg); - syncAppendEntriesDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_APPEND_ENTRIES_REPLY) { - SyncAppendEntriesReply* pSyncMsg = syncAppendEntriesReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncAppendEntriesReply2Json(pSyncMsg); - syncAppendEntriesReplyDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_SNAPSHOT_SEND) { - SyncSnapshotSend* pSyncMsg = syncSnapshotSendDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncSnapshotSend2Json(pSyncMsg); - syncSnapshotSendDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_SNAPSHOT_RSP) { - SyncSnapshotRsp* pSyncMsg = syncSnapshotRspDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncSnapshotRsp2Json(pSyncMsg); - syncSnapshotRspDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_LEADER_TRANSFER) { - SyncLeaderTransfer* pSyncMsg = syncLeaderTransferDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - pRoot = syncLeaderTransfer2Json(pSyncMsg); - syncLeaderTransferDestroy(pSyncMsg); - - } else if (pRpcMsg->msgType == TDMT_SYNC_COMMON_RESPONSE) { - pRoot = cJSON_CreateObject(); - char* s; - s = syncUtilPrintBin((char*)(pRpcMsg->pCont), pRpcMsg->contLen); - cJSON_AddStringToObject(pRoot, "pCont", s); - taosMemoryFree(s); - s = syncUtilPrintBin2((char*)(pRpcMsg->pCont), pRpcMsg->contLen); - cJSON_AddStringToObject(pRoot, "pCont2", s); - taosMemoryFree(s); - - } else { - pRoot = cJSON_CreateObject(); - char* s; - s = syncUtilPrintBin((char*)(pRpcMsg->pCont), pRpcMsg->contLen); - cJSON_AddStringToObject(pRoot, "pCont", s); - taosMemoryFree(s); - s = syncUtilPrintBin2((char*)(pRpcMsg->pCont), pRpcMsg->contLen); - cJSON_AddStringToObject(pRoot, "pCont2", s); - taosMemoryFree(s); - } - - cJSON_AddNumberToObject(pRoot, "msgType", pRpcMsg->msgType); - cJSON_AddNumberToObject(pRoot, "contLen", pRpcMsg->contLen); - cJSON_AddNumberToObject(pRoot, "code", pRpcMsg->code); - // cJSON_AddNumberToObject(pRoot, "persist", pRpcMsg->persist); - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "RpcMsg", pRoot); - return pJson; -} - -cJSON* syncRpcUnknownMsg2Json() { - cJSON* pRoot = cJSON_CreateObject(); - cJSON_AddNumberToObject(pRoot, "msgType", TDMT_SYNC_UNKNOWN); - cJSON_AddStringToObject(pRoot, "data", "unknown message"); - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SyncUnknown", pRoot); - return pJson; -} - -char* syncRpcMsg2Str(SRpcMsg* pRpcMsg) { - cJSON* pJson = syncRpcMsg2Json(pRpcMsg); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -// for debug ---------------------- -void syncRpcMsgPrint(SRpcMsg* pMsg) { - char* serialized = syncRpcMsg2Str(pMsg); - printf("syncRpcMsgPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncRpcMsgPrint2(char* s, SRpcMsg* pMsg) { - char* serialized = syncRpcMsg2Str(pMsg); - printf("syncRpcMsgPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncRpcMsgLog(SRpcMsg* pMsg) { - char* serialized = syncRpcMsg2Str(pMsg); - sTrace("syncRpcMsgLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncRpcMsgLog2(char* s, SRpcMsg* pMsg) { - if (gRaftDetailLog) { - char* serialized = syncRpcMsg2Str(pMsg); - sTrace("syncRpcMsgLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - // ---- message process SyncTimeout---- SyncTimeout* syncTimeoutBuild() { uint32_t bytes = sizeof(SyncTimeout); @@ -840,69 +700,49 @@ SyncClientRequest* syncClientRequestAlloc(uint32_t dataLen) { return pMsg; } -// step 1. original SRpcMsg => SyncClientRequest, add seqNum, isWeak -SyncClientRequest* syncClientRequestBuild(const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, bool isWeak, int32_t vgId) { - SyncClientRequest* pMsg = syncClientRequestAlloc(pOriginalRpcMsg->contLen); - pMsg->vgId = vgId; - pMsg->originalRpcType = pOriginalRpcMsg->msgType; - pMsg->seqNum = seqNum; - pMsg->isWeak = isWeak; - memcpy(pMsg->data, pOriginalRpcMsg->pCont, pOriginalRpcMsg->contLen); - return pMsg; -} - -void syncClientRequestDestroy(SyncClientRequest* pMsg) { - if (pMsg != NULL) { - taosMemoryFree(pMsg); +int32_t syncClientRequestBuildFromRpcMsg(SRpcMsg* pClientRequestRpcMsg, const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, + bool isWeak, int32_t vgId) { + int32_t bytes = sizeof(SyncClientRequest) + pOriginalRpcMsg->contLen; + pClientRequestRpcMsg->pCont = rpcMallocCont(bytes); + if (pClientRequestRpcMsg->pCont == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } -} -void syncClientRequestSerialize(const SyncClientRequest* pMsg, char* buf, uint32_t bufLen) { - ASSERT(pMsg->bytes <= bufLen); - memcpy(buf, pMsg, pMsg->bytes); -} + SyncClientRequest* pClientRequest = pClientRequestRpcMsg->pCont; + pClientRequest->bytes = bytes; + pClientRequest->vgId = vgId; + pClientRequest->msgType = TDMT_SYNC_CLIENT_REQUEST; + pClientRequest->originalRpcType = pOriginalRpcMsg->msgType; + pClientRequest->seqNum = seqNum; + pClientRequest->isWeak = isWeak; + pClientRequest->dataLen = pOriginalRpcMsg->contLen; + memcpy(pClientRequest->data, (char*)pOriginalRpcMsg->pCont, pOriginalRpcMsg->contLen); -void syncClientRequestDeserialize(const char* buf, uint32_t len, SyncClientRequest* pMsg) { - memcpy(pMsg, buf, len); - ASSERT(len == pMsg->bytes); + pClientRequestRpcMsg->msgType = TDMT_SYNC_CLIENT_REQUEST; + pClientRequestRpcMsg->contLen = bytes; + return 0; } -char* syncClientRequestSerialize2(const SyncClientRequest* pMsg, uint32_t* len) { - char* buf = taosMemoryMalloc(pMsg->bytes); - ASSERT(buf != NULL); - syncClientRequestSerialize(pMsg, buf, pMsg->bytes); - if (len != NULL) { - *len = pMsg->bytes; +int32_t syncClientRequestBuildFromNoopEntry(SRpcMsg* pClientRequestRpcMsg, const SSyncRaftEntry* pEntry, int32_t vgId) { + int32_t bytes = sizeof(SyncClientRequest) + pEntry->bytes; + pClientRequestRpcMsg->pCont = rpcMallocCont(bytes); + if (pClientRequestRpcMsg->pCont == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } - return buf; -} - -SyncClientRequest* syncClientRequestDeserialize2(const char* buf, uint32_t len) { - uint32_t bytes = *((uint32_t*)buf); - SyncClientRequest* pMsg = taosMemoryMalloc(bytes); - ASSERT(pMsg != NULL); - syncClientRequestDeserialize(buf, len, pMsg); - ASSERT(len == pMsg->bytes); - return pMsg; -} - -// step 2. SyncClientRequest => RpcMsg, to queue -void syncClientRequest2RpcMsg(const SyncClientRequest* pMsg, SRpcMsg* pRpcMsg) { - pRpcMsg->msgType = pMsg->msgType; - pRpcMsg->contLen = pMsg->bytes; - pRpcMsg->pCont = rpcMallocCont(pRpcMsg->contLen); - syncClientRequestSerialize(pMsg, pRpcMsg->pCont, pRpcMsg->contLen); -} -void syncClientRequestFromRpcMsg(const SRpcMsg* pRpcMsg, SyncClientRequest* pMsg) { - syncClientRequestDeserialize(pRpcMsg->pCont, pRpcMsg->contLen, pMsg); -} + SyncClientRequest* pClientRequest = pClientRequestRpcMsg->pCont; + pClientRequest->bytes = bytes; + pClientRequest->vgId = vgId; + pClientRequest->msgType = TDMT_SYNC_CLIENT_REQUEST; + pClientRequest->originalRpcType = TDMT_SYNC_NOOP; + pClientRequest->dataLen = pEntry->bytes; + memcpy(pClientRequest->data, (char*)pEntry, pEntry->bytes); -// step 3. RpcMsg => SyncClientRequest, from queue -SyncClientRequest* syncClientRequestFromRpcMsg2(const SRpcMsg* pRpcMsg) { - SyncClientRequest* pMsg = syncClientRequestDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); - ASSERT(pMsg != NULL); - return pMsg; + pClientRequestRpcMsg->msgType = TDMT_SYNC_CLIENT_REQUEST; + pClientRequestRpcMsg->contLen = bytes; + return 0; } cJSON* syncClientRequest2Json(const SyncClientRequest* pMsg) { @@ -940,35 +780,6 @@ char* syncClientRequest2Str(const SyncClientRequest* pMsg) { return serialized; } -// for debug ---------------------- -void syncClientRequestPrint(const SyncClientRequest* pMsg) { - char* serialized = syncClientRequest2Str(pMsg); - printf("syncClientRequestPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncClientRequestPrint2(char* s, const SyncClientRequest* pMsg) { - char* serialized = syncClientRequest2Str(pMsg); - printf("syncClientRequestPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncClientRequestLog(const SyncClientRequest* pMsg) { - char* serialized = syncClientRequest2Str(pMsg); - sTrace("syncClientRequestLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncClientRequestLog2(char* s, const SyncClientRequest* pMsg) { - if (gRaftDetailLog) { - char* serialized = syncClientRequest2Str(pMsg); - sTrace("syncClientRequestLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - // ---- message process SyncClientRequestBatch---- // block1: @@ -1059,86 +870,6 @@ SyncClientRequestBatch* syncClientRequestBatchFromRpcMsg(const SRpcMsg* pRpcMsg) return pSyncMsg; } -cJSON* syncClientRequestBatch2Json(const SyncClientRequestBatch* pMsg) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pMsg != NULL) { - cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); - cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); - cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); - cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); - cJSON_AddNumberToObject(pRoot, "dataCount", pMsg->dataCount); - - SRaftMeta* metaArr = syncClientRequestBatchMetaArr(pMsg); - SRpcMsg* msgArr = syncClientRequestBatchRpcMsgArr(pMsg); - - cJSON* pMetaArr = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "metaArr", pMetaArr); - for (int i = 0; i < pMsg->dataCount; ++i) { - cJSON* pMeta = cJSON_CreateObject(); - cJSON_AddNumberToObject(pMeta, "seqNum", metaArr[i].seqNum); - cJSON_AddNumberToObject(pMeta, "isWeak", metaArr[i].isWeak); - cJSON_AddItemToArray(pMetaArr, pMeta); - } - - cJSON* pMsgArr = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "msgArr", pMsgArr); - for (int i = 0; i < pMsg->dataCount; ++i) { - cJSON* pRpcMsgJson = syncRpcMsg2Json(&msgArr[i]); - cJSON_AddItemToArray(pMsgArr, pRpcMsgJson); - } - - char* s; - s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data", s); - taosMemoryFree(s); - s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data2", s); - taosMemoryFree(s); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SyncClientRequestBatch", pRoot); - return pJson; -} - -char* syncClientRequestBatch2Str(const SyncClientRequestBatch* pMsg) { - cJSON* pJson = syncClientRequestBatch2Json(pMsg); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -// for debug ---------------------- -void syncClientRequestBatchPrint(const SyncClientRequestBatch* pMsg) { - char* serialized = syncClientRequestBatch2Str(pMsg); - printf("syncClientRequestBatchPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncClientRequestBatchPrint2(char* s, const SyncClientRequestBatch* pMsg) { - char* serialized = syncClientRequestBatch2Str(pMsg); - printf("syncClientRequestBatchPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncClientRequestBatchLog(const SyncClientRequestBatch* pMsg) { - char* serialized = syncClientRequestBatch2Str(pMsg); - sTrace("syncClientRequestBatchLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncClientRequestBatchLog2(char* s, const SyncClientRequestBatch* pMsg) { - if (gRaftDetailLog) { - char* serialized = syncClientRequestBatch2Str(pMsg); - sLTrace("syncClientRequestBatchLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - // ---- message process SyncRequestVote---- SyncRequestVote* syncRequestVoteBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncRequestVote); @@ -1716,138 +1447,6 @@ SyncAppendEntriesBatch* syncAppendEntriesBatchFromRpcMsg2(const SRpcMsg* pRpcMsg return pMsg; } -cJSON* syncAppendEntriesBatch2Json(const SyncAppendEntriesBatch* pMsg) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pMsg != NULL) { - cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); - cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); - cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); - - cJSON* pSrcId = cJSON_CreateObject(); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->srcId.addr); - cJSON_AddStringToObject(pSrcId, "addr", u64buf); - { - uint64_t u64 = pMsg->srcId.addr; - cJSON* pTmp = pSrcId; - char host[128] = {0}; - uint16_t port; - syncUtilU642Addr(u64, host, sizeof(host), &port); - cJSON_AddStringToObject(pTmp, "addr_host", host); - cJSON_AddNumberToObject(pTmp, "addr_port", port); - } - cJSON_AddNumberToObject(pSrcId, "vgId", pMsg->srcId.vgId); - cJSON_AddItemToObject(pRoot, "srcId", pSrcId); - - cJSON* pDestId = cJSON_CreateObject(); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->destId.addr); - cJSON_AddStringToObject(pDestId, "addr", u64buf); - { - uint64_t u64 = pMsg->destId.addr; - cJSON* pTmp = pDestId; - char host[128] = {0}; - uint16_t port; - syncUtilU642Addr(u64, host, sizeof(host), &port); - cJSON_AddStringToObject(pTmp, "addr_host", host); - cJSON_AddNumberToObject(pTmp, "addr_port", port); - } - cJSON_AddNumberToObject(pDestId, "vgId", pMsg->destId.vgId); - cJSON_AddItemToObject(pRoot, "destId", pDestId); - - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term); - cJSON_AddStringToObject(pRoot, "term", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->prevLogIndex); - cJSON_AddStringToObject(pRoot, "prevLogIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->prevLogTerm); - cJSON_AddStringToObject(pRoot, "prevLogTerm", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->commitIndex); - cJSON_AddStringToObject(pRoot, "commitIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->privateTerm); - cJSON_AddStringToObject(pRoot, "privateTerm", u64buf); - - cJSON_AddNumberToObject(pRoot, "dataCount", pMsg->dataCount); - cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); - - int32_t metaArrayLen = sizeof(SOffsetAndContLen) * pMsg->dataCount; // - int32_t entryArrayLen = pMsg->dataLen - metaArrayLen; - - cJSON_AddNumberToObject(pRoot, "metaArrayLen", metaArrayLen); - cJSON_AddNumberToObject(pRoot, "entryArrayLen", entryArrayLen); - - SOffsetAndContLen* metaArr = (SOffsetAndContLen*)(pMsg->data); - - cJSON* pMetaArr = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "metaArr", pMetaArr); - for (int i = 0; i < pMsg->dataCount; ++i) { - cJSON* pMeta = cJSON_CreateObject(); - cJSON_AddNumberToObject(pMeta, "offset", metaArr[i].offset); - cJSON_AddNumberToObject(pMeta, "contLen", metaArr[i].contLen); - cJSON_AddItemToArray(pMetaArr, pMeta); - } - - cJSON* pEntryArr = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "entryArr", pEntryArr); - for (int i = 0; i < pMsg->dataCount; ++i) { - SSyncRaftEntry* pEntry = (SSyncRaftEntry*)(pMsg->data + metaArr[i].offset); - cJSON* pEntryJson = syncEntry2Json(pEntry); - cJSON_AddItemToArray(pEntryArr, pEntryJson); - } - - char* s; - s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data", s); - taosMemoryFree(s); - s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data2", s); - taosMemoryFree(s); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SyncAppendEntriesBatch", pRoot); - return pJson; -} - -char* syncAppendEntriesBatch2Str(const SyncAppendEntriesBatch* pMsg) { - cJSON* pJson = syncAppendEntriesBatch2Json(pMsg); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -// for debug ---------------------- -void syncAppendEntriesBatchPrint(const SyncAppendEntriesBatch* pMsg) { - char* serialized = syncAppendEntriesBatch2Str(pMsg); - printf("syncAppendEntriesBatchPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncAppendEntriesBatchPrint2(char* s, const SyncAppendEntriesBatch* pMsg) { - char* serialized = syncAppendEntriesBatch2Str(pMsg); - printf("syncAppendEntriesBatchPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncAppendEntriesBatchLog(const SyncAppendEntriesBatch* pMsg) { - char* serialized = syncAppendEntriesBatch2Str(pMsg); - sTrace("syncAppendEntriesBatchLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncAppendEntriesBatchLog2(char* s, const SyncAppendEntriesBatch* pMsg) { - if (gRaftDetailLog) { - char* serialized = syncAppendEntriesBatch2Str(pMsg); - sLTrace("syncAppendEntriesBatchLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - // ---- message process SyncAppendEntriesReply---- SyncAppendEntriesReply* syncAppendEntriesReplyBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncAppendEntriesReply); @@ -2827,118 +2426,6 @@ SyncSnapshotSend* syncSnapshotSendFromRpcMsg2(const SRpcMsg* pRpcMsg) { return pMsg; } -cJSON* syncSnapshotSend2Json(const SyncSnapshotSend* pMsg) { - char u64buf[128]; - cJSON* pRoot = cJSON_CreateObject(); - - if (pMsg != NULL) { - cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); - cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); - cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); - - cJSON* pSrcId = cJSON_CreateObject(); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->srcId.addr); - cJSON_AddStringToObject(pSrcId, "addr", u64buf); - { - uint64_t u64 = pMsg->srcId.addr; - cJSON* pTmp = pSrcId; - char host[128]; - uint16_t port; - syncUtilU642Addr(u64, host, sizeof(host), &port); - cJSON_AddStringToObject(pTmp, "addr_host", host); - cJSON_AddNumberToObject(pTmp, "addr_port", port); - } - cJSON_AddNumberToObject(pSrcId, "vgId", pMsg->srcId.vgId); - cJSON_AddItemToObject(pRoot, "srcId", pSrcId); - - cJSON* pDestId = cJSON_CreateObject(); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->destId.addr); - cJSON_AddStringToObject(pDestId, "addr", u64buf); - { - uint64_t u64 = pMsg->destId.addr; - cJSON* pTmp = pDestId; - char host[128]; - uint16_t port; - syncUtilU642Addr(u64, host, sizeof(host), &port); - cJSON_AddStringToObject(pTmp, "addr_host", host); - cJSON_AddNumberToObject(pTmp, "addr_port", port); - } - cJSON_AddNumberToObject(pDestId, "vgId", pMsg->destId.vgId); - cJSON_AddItemToObject(pRoot, "destId", pDestId); - - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term); - cJSON_AddStringToObject(pRoot, "term", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->startTime); - cJSON_AddStringToObject(pRoot, "startTime", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->beginIndex); - cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastIndex); - cJSON_AddStringToObject(pRoot, "lastIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastConfigIndex); - cJSON_AddStringToObject(pRoot, "lastConfigIndex", u64buf); - cJSON_AddItemToObject(pRoot, "lastConfig", syncCfg2Json((SSyncCfg*)&(pMsg->lastConfig))); - - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->lastTerm); - cJSON_AddStringToObject(pRoot, "lastTerm", u64buf); - - cJSON_AddNumberToObject(pRoot, "seq", pMsg->seq); - - cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); - char* s; - s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data", s); - taosMemoryFree(s); - s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); - cJSON_AddStringToObject(pRoot, "data2", s); - taosMemoryFree(s); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SyncSnapshotSend", pRoot); - return pJson; -} - -char* syncSnapshotSend2Str(const SyncSnapshotSend* pMsg) { - cJSON* pJson = syncSnapshotSend2Json(pMsg); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -// for debug ---------------------- -void syncSnapshotSendPrint(const SyncSnapshotSend* pMsg) { - char* serialized = syncSnapshotSend2Str(pMsg); - printf("syncSnapshotSendPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncSnapshotSendPrint2(char* s, const SyncSnapshotSend* pMsg) { - char* serialized = syncSnapshotSend2Str(pMsg); - printf("syncSnapshotSendPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncSnapshotSendLog(const SyncSnapshotSend* pMsg) { - char* serialized = syncSnapshotSend2Str(pMsg); - sTrace("syncSnapshotSendLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncSnapshotSendLog2(char* s, const SyncSnapshotSend* pMsg) { - if (gRaftDetailLog) { - char* serialized = syncSnapshotSend2Str(pMsg); - sTrace("syncSnapshotSendLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - -// --------------------------------------------- SyncSnapshotRsp* syncSnapshotRspBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncSnapshotRsp); SyncSnapshotRsp* pMsg = taosMemoryMalloc(bytes); diff --git a/source/libs/sync/src/syncRaftEntry.c b/source/libs/sync/src/syncRaftEntry.c index 520ecd95db98d98543c20b75347891318049c14d..c5b1399c8caf8b53eeb7ca16c1f505ef575e7ea4 100644 --- a/source/libs/sync/src/syncRaftEntry.c +++ b/source/libs/sync/src/syncRaftEntry.c @@ -13,31 +13,28 @@ * along with this program. If not, see . */ +#define _DEFAULT_SOURCE #include "syncRaftEntry.h" #include "syncUtil.h" -SSyncRaftEntry* syncEntryBuild(uint32_t dataLen) { - uint32_t bytes = sizeof(SSyncRaftEntry) + dataLen; +SSyncRaftEntry* syncEntryBuild(int32_t dataLen) { + int32_t bytes = sizeof(SSyncRaftEntry) + dataLen; SSyncRaftEntry* pEntry = taosMemoryMalloc(bytes); - ASSERT(pEntry != NULL); - memset(pEntry, 0, bytes); + if (pEntry == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + pEntry->bytes = bytes; pEntry->dataLen = dataLen; pEntry->rid = -1; - return pEntry; -} - -// step 4. SyncClientRequest => SSyncRaftEntry, add term, index -SSyncRaftEntry* syncEntryBuild2(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index) { - SSyncRaftEntry* pEntry = syncEntryBuild3(pMsg, term, index); - ASSERT(pEntry != NULL); return pEntry; } -SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index) { +SSyncRaftEntry* syncEntryBuildFromClientRequest(const SyncClientRequest* pMsg, SyncTerm term, SyncIndex index) { SSyncRaftEntry* pEntry = syncEntryBuild(pMsg->dataLen); - ASSERT(pEntry != NULL); + if (pEntry == NULL) return NULL; pEntry->msgType = pMsg->msgType; pEntry->originalRpcType = pMsg->originalRpcType; @@ -45,42 +42,37 @@ SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncInde pEntry->isWeak = pMsg->isWeak; pEntry->term = term; pEntry->index = index; - pEntry->dataLen = pMsg->dataLen; memcpy(pEntry->data, pMsg->data, pMsg->dataLen); return pEntry; } -SSyncRaftEntry* syncEntryBuild4(SRpcMsg* pOriginalMsg, SyncTerm term, SyncIndex index) { - SSyncRaftEntry* pEntry = syncEntryBuild(pOriginalMsg->contLen); - ASSERT(pEntry != NULL); +SSyncRaftEntry* syncEntryBuildFromRpcMsg(const SRpcMsg* pMsg, SyncTerm term, SyncIndex index) { + SSyncRaftEntry* pEntry = syncEntryBuild(pMsg->contLen); + if (pEntry == NULL) return NULL; pEntry->msgType = TDMT_SYNC_CLIENT_REQUEST; - pEntry->originalRpcType = pOriginalMsg->msgType; + pEntry->originalRpcType = pMsg->msgType; pEntry->seqNum = 0; pEntry->isWeak = 0; pEntry->term = term; pEntry->index = index; - pEntry->dataLen = pOriginalMsg->contLen; - memcpy(pEntry->data, pOriginalMsg->pCont, pOriginalMsg->contLen); + memcpy(pEntry->data, pMsg->pCont, pMsg->contLen); + + return pEntry; +} + +SSyncRaftEntry* syncEntryBuildFromAppendEntries(const SyncAppendEntries* pMsg) { + SSyncRaftEntry* pEntry = syncEntryBuild(pMsg->dataLen); + if (pEntry == NULL) return NULL; + memcpy(pEntry, pMsg->data, pMsg->dataLen); return pEntry; } SSyncRaftEntry* syncEntryBuildNoop(SyncTerm term, SyncIndex index, int32_t vgId) { - // init rpcMsg - SMsgHead head; - head.vgId = vgId; - head.contLen = sizeof(SMsgHead); - SRpcMsg rpcMsg; - memset(&rpcMsg, 0, sizeof(SRpcMsg)); - rpcMsg.contLen = head.contLen; - rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); - rpcMsg.msgType = TDMT_SYNC_NOOP; - memcpy(rpcMsg.pCont, &head, sizeof(head)); - - SSyncRaftEntry* pEntry = syncEntryBuild(rpcMsg.contLen); - ASSERT(pEntry != NULL); + SSyncRaftEntry* pEntry = syncEntryBuild(sizeof(SMsgHead)); + if (pEntry == NULL) return NULL; pEntry->msgType = TDMT_SYNC_CLIENT_REQUEST; pEntry->originalRpcType = TDMT_SYNC_NOOP; @@ -89,9 +81,9 @@ SSyncRaftEntry* syncEntryBuildNoop(SyncTerm term, SyncIndex index, int32_t vgId) pEntry->term = term; pEntry->index = index; - ASSERT(pEntry->dataLen == rpcMsg.contLen); - memcpy(pEntry->data, rpcMsg.pCont, rpcMsg.contLen); - rpcFreeCont(rpcMsg.pCont); + SMsgHead* pHead = (SMsgHead*)pEntry->data; + pHead->vgId = vgId; + pHead->contLen = sizeof(SMsgHead); return pEntry; } @@ -102,104 +94,13 @@ void syncEntryDestory(SSyncRaftEntry* pEntry) { } } -// step 5. SSyncRaftEntry => bin, to raft log -char* syncEntrySerialize(const SSyncRaftEntry* pEntry, uint32_t* len) { - char* buf = taosMemoryMalloc(pEntry->bytes); - ASSERT(buf != NULL); - memcpy(buf, pEntry, pEntry->bytes); - if (len != NULL) { - *len = pEntry->bytes; - } - return buf; -} - -// step 6. bin => SSyncRaftEntry, from raft log -SSyncRaftEntry* syncEntryDeserialize(const char* buf, uint32_t len) { - uint32_t bytes = *((uint32_t*)buf); - SSyncRaftEntry* pEntry = taosMemoryMalloc(bytes); - ASSERT(pEntry != NULL); - memcpy(pEntry, buf, len); - ASSERT(len == pEntry->bytes); - return pEntry; -} - -cJSON* syncEntry2Json(const SSyncRaftEntry* pEntry) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pEntry != NULL) { - cJSON_AddNumberToObject(pRoot, "bytes", pEntry->bytes); - cJSON_AddNumberToObject(pRoot, "msgType", pEntry->msgType); - cJSON_AddNumberToObject(pRoot, "originalRpcType", pEntry->originalRpcType); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->seqNum); - cJSON_AddStringToObject(pRoot, "seqNum", u64buf); - cJSON_AddNumberToObject(pRoot, "isWeak", pEntry->isWeak); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->term); - cJSON_AddStringToObject(pRoot, "term", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->index); - cJSON_AddStringToObject(pRoot, "index", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->rid); - cJSON_AddStringToObject(pRoot, "rid", u64buf); - cJSON_AddNumberToObject(pRoot, "dataLen", pEntry->dataLen); - - char* s; - s = syncUtilPrintBin((char*)(pEntry->data), pEntry->dataLen); - cJSON_AddStringToObject(pRoot, "data", s); - taosMemoryFree(s); - - s = syncUtilPrintBin2((char*)(pEntry->data), pEntry->dataLen); - cJSON_AddStringToObject(pRoot, "data2", s); - taosMemoryFree(s); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SSyncRaftEntry", pRoot); - return pJson; -} - -char* syncEntry2Str(const SSyncRaftEntry* pEntry) { - cJSON* pJson = syncEntry2Json(pEntry); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -// step 7. SSyncRaftEntry => original SRpcMsg, commit to user, delete seqNum, isWeak, term, index void syncEntry2OriginalRpc(const SSyncRaftEntry* pEntry, SRpcMsg* pRpcMsg) { - memset(pRpcMsg, 0, sizeof(*pRpcMsg)); pRpcMsg->msgType = pEntry->originalRpcType; pRpcMsg->contLen = pEntry->dataLen; pRpcMsg->pCont = rpcMallocCont(pRpcMsg->contLen); memcpy(pRpcMsg->pCont, pEntry->data, pRpcMsg->contLen); } -// for debug ---------------------- -void syncEntryPrint(const SSyncRaftEntry* pObj) { - char* serialized = syncEntry2Str(pObj); - printf("syncEntryPrint | len:%zu | %s \n", strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncEntryPrint2(char* s, const SSyncRaftEntry* pObj) { - char* serialized = syncEntry2Str(pObj); - printf("syncEntryPrint2 | len:%zu | %s | %s \n", strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void syncEntryLog(const SSyncRaftEntry* pObj) { - char* serialized = syncEntry2Str(pObj); - sTrace("syncEntryLog | len:%zu | %s", strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void syncEntryLog2(char* s, const SSyncRaftEntry* pObj) { - char* serialized = syncEntry2Str(pObj); - sTrace("syncEntryLog2 | len:%zu | %s | %s", strlen(serialized), s, serialized); - taosMemoryFree(serialized); -} - //----------------------------------- SRaftEntryHashCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount) { SRaftEntryHashCache* pCache = taosMemoryMalloc(sizeof(SRaftEntryHashCache)); @@ -354,76 +255,6 @@ int32_t raftCacheClear(struct SRaftEntryHashCache* pCache) { return 0; } -//----------------------------------- -cJSON* raftCache2Json(SRaftEntryHashCache* pCache) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pCache != NULL) { - taosThreadMutexLock(&pCache->mutex); - - snprintf(u64buf, sizeof(u64buf), "%p", pCache->pSyncNode); - cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); - cJSON_AddNumberToObject(pRoot, "currentCount", pCache->currentCount); - cJSON_AddNumberToObject(pRoot, "maxCount", pCache->maxCount); - cJSON* pEntries = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "entries", pEntries); - - SSyncRaftEntry* pIter = (SSyncRaftEntry*)taosHashIterate(pCache->pEntryHash, NULL); - if (pIter != NULL) { - SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pIter; - cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); - } - while (pIter) { - pIter = taosHashIterate(pCache->pEntryHash, pIter); - if (pIter != NULL) { - SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pIter; - cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); - } - } - - taosThreadMutexUnlock(&pCache->mutex); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SRaftEntryHashCache", pRoot); - return pJson; -} - -char* raftCache2Str(SRaftEntryHashCache* pCache) { - cJSON* pJson = raftCache2Json(pCache); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -void raftCachePrint(SRaftEntryHashCache* pCache) { - char* serialized = raftCache2Str(pCache); - printf("raftCachePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void raftCachePrint2(char* s, SRaftEntryHashCache* pCache) { - char* serialized = raftCache2Str(pCache); - printf("raftCachePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void raftCacheLog(SRaftEntryHashCache* pCache) { - char* serialized = raftCache2Str(pCache); - sTrace("raftCacheLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void raftCacheLog2(char* s, SRaftEntryHashCache* pCache) { - if (gRaftDetailLog) { - char* serialized = raftCache2Str(pCache); - sLTrace("raftCacheLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} //----------------------------------- static char* keyFn(const void* pData) { @@ -612,69 +443,3 @@ int32_t raftEntryCacheClear(struct SRaftEntryCache* pCache, int32_t count) { taosThreadMutexUnlock(&pCache->mutex); return returnCnt; } - -cJSON* raftEntryCache2Json(SRaftEntryCache* pCache) { - char u64buf[128] = {0}; - cJSON* pRoot = cJSON_CreateObject(); - - if (pCache != NULL) { - taosThreadMutexLock(&pCache->mutex); - - snprintf(u64buf, sizeof(u64buf), "%p", pCache->pSyncNode); - cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); - cJSON_AddNumberToObject(pRoot, "currentCount", pCache->currentCount); - cJSON_AddNumberToObject(pRoot, "maxCount", pCache->maxCount); - cJSON* pEntries = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "entries", pEntries); - - SSkipListIterator* pIter = tSkipListCreateIter(pCache->pSkipList); - while (tSkipListIterNext(pIter)) { - SSkipListNode* pNode = tSkipListIterGet(pIter); - ASSERT(pNode != NULL); - SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode); - cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); - } - tSkipListDestroyIter(pIter); - - taosThreadMutexUnlock(&pCache->mutex); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SRaftEntryCache", pRoot); - return pJson; -} - -char* raftEntryCache2Str(SRaftEntryCache* pObj) { - cJSON* pJson = raftEntryCache2Json(pObj); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -void raftEntryCachePrint(SRaftEntryCache* pObj) { - char* serialized = raftEntryCache2Str(pObj); - printf("raftEntryCachePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj) { - char* serialized = raftEntryCache2Str(pObj); - printf("raftEntryCachePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void raftEntryCacheLog(SRaftEntryCache* pObj) { - char* serialized = raftEntryCache2Str(pObj); - sTrace("raftEntryCacheLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj) { - if (gRaftDetailLog) { - char* serialized = raftEntryCache2Str(pObj); - sLTrace("raftEntryCacheLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} diff --git a/source/libs/sync/src/syncRaftLog.c b/source/libs/sync/src/syncRaftLog.c index d3d69b288b1baa9070fb6605602ca37598e4d226..85c94884f20e84c6a5ed163a4fea518928991721 100644 --- a/source/libs/sync/src/syncRaftLog.c +++ b/source/libs/sync/src/syncRaftLog.c @@ -22,15 +22,8 @@ // public function static int32_t raftLogRestoreFromSnapshot(struct SSyncLogStore* pLogStore, SyncIndex snapshotIndex); -static SyncIndex raftLogBeginIndex(struct SSyncLogStore* pLogStore); -static SyncIndex raftLogEndIndex(struct SSyncLogStore* pLogStore); -static SyncIndex raftLogWriteIndex(struct SSyncLogStore* pLogStore); -static bool raftLogIsEmpty(struct SSyncLogStore* pLogStore); -static int32_t raftLogEntryCount(struct SSyncLogStore* pLogStore); -static SyncIndex raftLogLastIndex(struct SSyncLogStore* pLogStore); -static SyncTerm raftLogLastTerm(struct SSyncLogStore* pLogStore); + static int32_t raftLogAppendEntry(struct SSyncLogStore* pLogStore, SSyncRaftEntry* pEntry); -static int32_t raftLogGetEntry(struct SSyncLogStore* pLogStore, SyncIndex index, SSyncRaftEntry** ppEntry); static int32_t raftLogTruncate(struct SSyncLogStore* pLogStore, SyncIndex fromIndex); static bool raftLogExist(struct SSyncLogStore* pLogStore, SyncIndex index); static int32_t raftLogUpdateCommitIndex(SSyncLogStore* pLogStore, SyncIndex index); @@ -126,29 +119,29 @@ static int32_t raftLogRestoreFromSnapshot(struct SSyncLogStore* pLogStore, SyncI return 0; } -static SyncIndex raftLogBeginIndex(struct SSyncLogStore* pLogStore) { +SyncIndex raftLogBeginIndex(struct SSyncLogStore* pLogStore) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; SyncIndex firstVer = walGetFirstVer(pWal); return firstVer; } -static SyncIndex raftLogEndIndex(struct SSyncLogStore* pLogStore) { return raftLogLastIndex(pLogStore); } +SyncIndex raftLogEndIndex(struct SSyncLogStore* pLogStore) { return raftLogLastIndex(pLogStore); } -static bool raftLogIsEmpty(struct SSyncLogStore* pLogStore) { +bool raftLogIsEmpty(struct SSyncLogStore* pLogStore) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; return walIsEmpty(pWal); } -static int32_t raftLogEntryCount(struct SSyncLogStore* pLogStore) { +int32_t raftLogEntryCount(struct SSyncLogStore* pLogStore) { SyncIndex beginIndex = raftLogBeginIndex(pLogStore); SyncIndex endIndex = raftLogEndIndex(pLogStore); int32_t count = endIndex - beginIndex + 1; return count > 0 ? count : 0; } -static SyncIndex raftLogLastIndex(struct SSyncLogStore* pLogStore) { +SyncIndex raftLogLastIndex(struct SSyncLogStore* pLogStore) { SyncIndex lastIndex; SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; @@ -157,7 +150,7 @@ static SyncIndex raftLogLastIndex(struct SSyncLogStore* pLogStore) { return lastVer; } -static SyncIndex raftLogWriteIndex(struct SSyncLogStore* pLogStore) { +SyncIndex raftLogWriteIndex(struct SSyncLogStore* pLogStore) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; SyncIndex lastVer = walGetLastVer(pWal); @@ -174,7 +167,7 @@ static bool raftLogExist(struct SSyncLogStore* pLogStore, SyncIndex index) { // if success, return last term // if not log, return 0 // if error, return SYNC_TERM_INVALID -static SyncTerm raftLogLastTerm(struct SSyncLogStore* pLogStore) { +SyncTerm raftLogLastTerm(struct SSyncLogStore* pLogStore) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; if (walIsEmpty(pWal)) { @@ -225,7 +218,7 @@ static int32_t raftLogAppendEntry(struct SSyncLogStore* pLogStore, SSyncRaftEntr // entry found, return 0 // entry not found, return -1, terrno = TSDB_CODE_WAL_LOG_NOT_EXIST // other error, return -1 -static int32_t raftLogGetEntry(struct SSyncLogStore* pLogStore, SyncIndex index, SSyncRaftEntry** ppEntry) { +int32_t raftLogGetEntry(struct SSyncLogStore* pLogStore, SyncIndex index, SSyncRaftEntry** ppEntry) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; int32_t code = 0; @@ -364,111 +357,6 @@ SyncIndex raftlogCommitIndex(SSyncLogStore* pLogStore) { return pData->pSyncNode->commitIndex; } -cJSON* logStore2Json(SSyncLogStore* pLogStore) { - char u64buf[128] = {0}; - SSyncLogStoreData* pData = (SSyncLogStoreData*)pLogStore->data; - cJSON* pRoot = cJSON_CreateObject(); - - if (pData != NULL && pData->pWal != NULL) { - snprintf(u64buf, sizeof(u64buf), "%p", pData->pSyncNode); - cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pData->pWal); - cJSON_AddStringToObject(pRoot, "pWal", u64buf); - - SyncIndex beginIndex = raftLogBeginIndex(pLogStore); - snprintf(u64buf, sizeof(u64buf), "%" PRId64, beginIndex); - cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); - - SyncIndex endIndex = raftLogEndIndex(pLogStore); - snprintf(u64buf, sizeof(u64buf), "%" PRId64, endIndex); - cJSON_AddStringToObject(pRoot, "endIndex", u64buf); - - int32_t count = raftLogEntryCount(pLogStore); - cJSON_AddNumberToObject(pRoot, "entryCount", count); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogWriteIndex(pLogStore)); - cJSON_AddStringToObject(pRoot, "WriteIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%d", raftLogIsEmpty(pLogStore)); - cJSON_AddStringToObject(pRoot, "IsEmpty", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogLastIndex(pLogStore)); - cJSON_AddStringToObject(pRoot, "LastIndex", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, raftLogLastTerm(pLogStore)); - cJSON_AddStringToObject(pRoot, "LastTerm", u64buf); - - cJSON* pEntries = cJSON_CreateArray(); - cJSON_AddItemToObject(pRoot, "pEntries", pEntries); - - if (!raftLogIsEmpty(pLogStore)) { - for (SyncIndex i = beginIndex; i <= endIndex; ++i) { - SSyncRaftEntry* pEntry = NULL; - raftLogGetEntry(pLogStore, i, &pEntry); - - cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); - syncEntryDestory(pEntry); - } - } - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SSyncLogStore", pRoot); - return pJson; -} - -char* logStore2Str(SSyncLogStore* pLogStore) { - cJSON* pJson = logStore2Json(pLogStore); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - -cJSON* logStoreSimple2Json(SSyncLogStore* pLogStore) { - char u64buf[128] = {0}; - SSyncLogStoreData* pData = (SSyncLogStoreData*)pLogStore->data; - cJSON* pRoot = cJSON_CreateObject(); - - if (pData != NULL && pData->pWal != NULL) { - snprintf(u64buf, sizeof(u64buf), "%p", pData->pSyncNode); - cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); - snprintf(u64buf, sizeof(u64buf), "%p", pData->pWal); - cJSON_AddStringToObject(pRoot, "pWal", u64buf); - - SyncIndex beginIndex = raftLogBeginIndex(pLogStore); - snprintf(u64buf, sizeof(u64buf), "%" PRId64, beginIndex); - cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); - - SyncIndex endIndex = raftLogEndIndex(pLogStore); - snprintf(u64buf, sizeof(u64buf), "%" PRId64, endIndex); - cJSON_AddStringToObject(pRoot, "endIndex", u64buf); - - int32_t count = raftLogEntryCount(pLogStore); - cJSON_AddNumberToObject(pRoot, "entryCount", count); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogWriteIndex(pLogStore)); - cJSON_AddStringToObject(pRoot, "WriteIndex", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%d", raftLogIsEmpty(pLogStore)); - cJSON_AddStringToObject(pRoot, "IsEmpty", u64buf); - - snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogLastIndex(pLogStore)); - cJSON_AddStringToObject(pRoot, "LastIndex", u64buf); - snprintf(u64buf, sizeof(u64buf), "%" PRIu64, raftLogLastTerm(pLogStore)); - cJSON_AddStringToObject(pRoot, "LastTerm", u64buf); - } - - cJSON* pJson = cJSON_CreateObject(); - cJSON_AddItemToObject(pJson, "SSyncLogStoreSimple", pRoot); - return pJson; -} - -char* logStoreSimple2Str(SSyncLogStore* pLogStore) { - cJSON* pJson = logStoreSimple2Json(pLogStore); - char* serialized = cJSON_Print(pJson); - cJSON_Delete(pJson); - return serialized; -} - SyncIndex logStoreFirstIndex(SSyncLogStore* pLogStore) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; @@ -480,63 +368,3 @@ SyncIndex logStoreWalCommitVer(SSyncLogStore* pLogStore) { SWal* pWal = pData->pWal; return walGetCommittedVer(pWal); } - -// for debug ----------------- -void logStorePrint(SSyncLogStore* pLogStore) { - char* serialized = logStore2Str(pLogStore); - printf("logStorePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void logStorePrint2(char* s, SSyncLogStore* pLogStore) { - char* serialized = logStore2Str(pLogStore); - printf("logStorePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void logStoreLog(SSyncLogStore* pLogStore) { - if (gRaftDetailLog) { - char* serialized = logStore2Str(pLogStore); - sLTrace("logStoreLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); - } -} - -void logStoreLog2(char* s, SSyncLogStore* pLogStore) { - if (gRaftDetailLog) { - char* serialized = logStore2Str(pLogStore); - sLTrace("logStoreLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} - -// for debug ----------------- -void logStoreSimplePrint(SSyncLogStore* pLogStore) { - char* serialized = logStoreSimple2Str(pLogStore); - printf("logStoreSimplePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void logStoreSimplePrint2(char* s, SSyncLogStore* pLogStore) { - char* serialized = logStoreSimple2Str(pLogStore); - printf("logStoreSimplePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); - fflush(NULL); - taosMemoryFree(serialized); -} - -void logStoreSimpleLog(SSyncLogStore* pLogStore) { - char* serialized = logStoreSimple2Str(pLogStore); - sTrace("logStoreSimpleLog | len:%d | %s", (int32_t)strlen(serialized), serialized); - taosMemoryFree(serialized); -} - -void logStoreSimpleLog2(char* s, SSyncLogStore* pLogStore) { - if (gRaftDetailLog) { - char* serialized = logStoreSimple2Str(pLogStore); - sTrace("logStoreSimpleLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); - taosMemoryFree(serialized); - } -} diff --git a/source/libs/sync/src/syncReplication.c b/source/libs/sync/src/syncReplication.c index 7b4ef634068382a89c516e9fc3919d37c6efaf0d..8c3f7e5384810929aa53390aeb3ef5376b632993 100644 --- a/source/libs/sync/src/syncReplication.c +++ b/source/libs/sync/src/syncReplication.c @@ -78,14 +78,7 @@ int32_t syncNodeReplicateOne(SSyncNode* pSyncNode, SRaftId* pDestId) { pMsg = syncAppendEntriesBuild(pEntry->bytes, pSyncNode->vgId); ASSERT(pMsg != NULL); - - // add pEntry into msg - uint32_t len; - char* serialized = syncEntrySerialize(pEntry, &len); - ASSERT(len == pEntry->bytes); - memcpy(pMsg->data, serialized, len); - - taosMemoryFree(serialized); + memcpy(pMsg->data, pEntry->data, pEntry->bytes); syncEntryDestory(pEntry); } else { diff --git a/source/libs/sync/test/CMakeLists.txt b/source/libs/sync/test/CMakeLists.txt index e584893ae0d3520f8c7739e473c13aff5a0fe5f8..7a22d96972eca3c6f73908cae302b2806df5eed1 100644 --- a/source/libs/sync/test/CMakeLists.txt +++ b/source/libs/sync/test/CMakeLists.txt @@ -1,3 +1,4 @@ +add_subdirectory(sync_test_lib) add_executable(syncTest "") add_executable(syncRaftIdCheck "") add_executable(syncEnvTest "") @@ -24,8 +25,6 @@ add_executable(syncRequestVoteReplyTest "") add_executable(syncAppendEntriesTest "") add_executable(syncAppendEntriesBatchTest "") add_executable(syncAppendEntriesReplyTest "") -add_executable(syncClientRequestTest "") -add_executable(syncClientRequestBatchTest "") add_executable(syncTimeoutTest "") add_executable(syncPingTest "") add_executable(syncPingReplyTest "") @@ -167,14 +166,6 @@ target_sources(syncAppendEntriesReplyTest PRIVATE "syncAppendEntriesReplyTest.cpp" ) -target_sources(syncClientRequestTest - PRIVATE - "syncClientRequestTest.cpp" -) -target_sources(syncClientRequestBatchTest - PRIVATE - "syncClientRequestBatchTest.cpp" -) target_sources(syncTimeoutTest PRIVATE "syncTimeoutTest.cpp" @@ -287,10 +278,6 @@ target_sources(syncLeaderTransferTest PRIVATE "syncLeaderTransferTest.cpp" ) -target_sources(syncReconfigFinishTest - PRIVATE - "syncReconfigFinishTest.cpp" -) target_sources(syncRestoreFromSnapshot PRIVATE "syncRestoreFromSnapshot.cpp" @@ -451,16 +438,6 @@ target_include_directories(syncAppendEntriesReplyTest "${TD_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) -target_include_directories(syncClientRequestTest - PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" - "${CMAKE_CURRENT_SOURCE_DIR}/../inc" -) -target_include_directories(syncClientRequestBatchTest - PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" - "${CMAKE_CURRENT_SOURCE_DIR}/../inc" -) target_include_directories(syncTimeoutTest PUBLIC "${TD_SOURCE_DIR}/include/libs/sync" @@ -601,11 +578,6 @@ target_include_directories(syncLeaderTransferTest "${TD_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) -target_include_directories(syncReconfigFinishTest - PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" - "${CMAKE_CURRENT_SOURCE_DIR}/../inc" -) target_include_directories(syncRestoreFromSnapshot PUBLIC "${TD_SOURCE_DIR}/include/libs/sync" @@ -644,259 +616,247 @@ target_include_directories(syncPreSnapshotReplyTest target_link_libraries(syncTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftIdCheck - sync + sync_test_lib gtest_main ) target_link_libraries(syncEnvTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPingTimerTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIOTickQTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIOTickPingTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIOSendMsgTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIOClientTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIOServerTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftStoreTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncEnqTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIndexTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncInitTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncUtilTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncVotesGrantedTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncVotesRespondTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncIndexMgrTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncLogStoreTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncEntryTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncEntryCacheTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncHashCacheTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRequestVoteTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRequestVoteReplyTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncAppendEntriesTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncAppendEntriesBatchTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncAppendEntriesReplyTest - sync - gtest_main -) -target_link_libraries(syncClientRequestTest - sync - gtest_main -) -target_link_libraries(syncClientRequestBatchTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncTimeoutTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPingTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPingReplyTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRpcMsgTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPingTimerTest2 - sync + sync_test_lib gtest_main ) target_link_libraries(syncPingSelfTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncElectTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncEncodeTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncWriteTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncReplicateTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRefTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncLogStoreCheck - sync + sync_test_lib gtest_main ) target_link_libraries(syncLogStoreCheck2 - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftCfgTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRespMgrTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncSnapshotTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncApplyMsgTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncConfigChangeTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncConfigChangeSnapshotTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncSnapshotSendTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncSnapshotRspTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncSnapshotSenderTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncSnapshotReceiverTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncTestTool - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftLogTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftLogTest2 - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftLogTest3 - sync + sync_test_lib gtest_main ) target_link_libraries(syncLeaderTransferTest - sync - gtest_main -) -target_link_libraries(syncReconfigFinishTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncRestoreFromSnapshot - sync + sync_test_lib gtest_main ) target_link_libraries(syncRaftCfgIndexTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncHeartbeatTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncHeartbeatReplyTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncLocalCmdTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPreSnapshotTest - sync + sync_test_lib gtest_main ) target_link_libraries(syncPreSnapshotReplyTest - sync + sync_test_lib gtest_main ) diff --git a/source/libs/sync/test/syncAppendEntriesBatchTest.cpp b/source/libs/sync/test/syncAppendEntriesBatchTest.cpp index f2544d8fec7afcf707207e792376488986834380..98b8654734e937384338e42ee7c404d298f887b2 100644 --- a/source/libs/sync/test/syncAppendEntriesBatchTest.cpp +++ b/source/libs/sync/test/syncAppendEntriesBatchTest.cpp @@ -6,6 +6,7 @@ #include "syncRaftEntry.h" #include "syncUtil.h" #include "trpc.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncClientRequestBatchTest.cpp b/source/libs/sync/test/syncClientRequestBatchTest.cpp deleted file mode 100644 index f07ee08b2bbb8805f1e9767d56d4ffb190d920f5..0000000000000000000000000000000000000000 --- a/source/libs/sync/test/syncClientRequestBatchTest.cpp +++ /dev/null @@ -1,125 +0,0 @@ -#include -#include -#include "syncIO.h" -#include "syncInt.h" -#include "syncMessage.h" -#include "syncUtil.h" - -void logTest() { - sTrace("--- sync log test: trace"); - sDebug("--- sync log test: debug"); - sInfo("--- sync log test: info"); - sWarn("--- sync log test: warn"); - sError("--- sync log test: error"); - sFatal("--- sync log test: fatal"); -} - -SRpcMsg *createRpcMsg(int32_t i, int32_t dataLen) { - SyncPing *pSyncMsg = syncPingBuild(20); - snprintf(pSyncMsg->data, pSyncMsg->dataLen, "value_%d", i); - - SRpcMsg *pRpcMsg = (SRpcMsg *)taosMemoryMalloc(sizeof(SRpcMsg)); - memset(pRpcMsg, 0, sizeof(SRpcMsg)); - pRpcMsg->code = 10 * i; - syncPing2RpcMsg(pSyncMsg, pRpcMsg); - - syncPingDestroy(pSyncMsg); - return pRpcMsg; -} - -SyncClientRequestBatch *createMsg() { - SRpcMsg *rpcMsgPArr[5]; - memset(rpcMsgPArr, 0, sizeof(rpcMsgPArr)); - for (int32_t i = 0; i < 5; ++i) { - SRpcMsg *pRpcMsg = createRpcMsg(i, 20); - rpcMsgPArr[i] = pRpcMsg; - // taosMemoryFree(pRpcMsg); - } - - SRaftMeta raftArr[5]; - memset(raftArr, 0, sizeof(raftArr)); - for (int32_t i = 0; i < 5; ++i) { - raftArr[i].seqNum = i * 10; - raftArr[i].isWeak = i % 2; - } - - SyncClientRequestBatch *pMsg = syncClientRequestBatchBuild(rpcMsgPArr, raftArr, 5, 1234); - return pMsg; -} - -void test1() { - SyncClientRequestBatch *pMsg = createMsg(); - syncClientRequestBatchLog2((char *)"==test1==", pMsg); - syncClientRequestBatchDestroyDeep(pMsg); -} - -/* -void test2() { - SyncClientRequest *pMsg = createMsg(); - uint32_t len = pMsg->bytes; - char * serialized = (char *)taosMemoryMalloc(len); - syncClientRequestSerialize(pMsg, serialized, len); - SyncClientRequest *pMsg2 = syncClientRequestAlloc(pMsg->dataLen); - syncClientRequestDeserialize(serialized, len, pMsg2); - syncClientRequestLog2((char *)"test2: syncClientRequestSerialize -> syncClientRequestDeserialize ", pMsg2); - - taosMemoryFree(serialized); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test3() { - SyncClientRequest *pMsg = createMsg(); - uint32_t len; - char * serialized = syncClientRequestSerialize2(pMsg, &len); - SyncClientRequest *pMsg2 = syncClientRequestDeserialize2(serialized, len); - syncClientRequestLog2((char *)"test3: syncClientRequestSerialize3 -> syncClientRequestDeserialize2 ", pMsg2); - - taosMemoryFree(serialized); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test4() { - SyncClientRequest *pMsg = createMsg(); - SRpcMsg rpcMsg; - syncClientRequest2RpcMsg(pMsg, &rpcMsg); - SyncClientRequest *pMsg2 = (SyncClientRequest *)taosMemoryMalloc(rpcMsg.contLen); - syncClientRequestFromRpcMsg(&rpcMsg, pMsg2); - syncClientRequestLog2((char *)"test4: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg ", pMsg2); - - rpcFreeCont(rpcMsg.pCont); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test5() { - SyncClientRequest *pMsg = createMsg(); - SRpcMsg rpcMsg; - syncClientRequest2RpcMsg(pMsg, &rpcMsg); - SyncClientRequest *pMsg2 = syncClientRequestFromRpcMsg2(&rpcMsg); - syncClientRequestLog2((char *)"test5: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg2 ", pMsg2); - - rpcFreeCont(rpcMsg.pCont); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} -*/ - -int main() { - gRaftDetailLog = true; - tsAsyncLog = 0; - sDebugFlag = DEBUG_DEBUG + DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; - logTest(); - - test1(); - - /* -test2(); -test3(); -test4(); -test5(); -*/ - - return 0; -} diff --git a/source/libs/sync/test/syncClientRequestTest.cpp b/source/libs/sync/test/syncClientRequestTest.cpp deleted file mode 100644 index b6bfcc2da5d672a92bc93fdd8aa859a8a4891c87..0000000000000000000000000000000000000000 --- a/source/libs/sync/test/syncClientRequestTest.cpp +++ /dev/null @@ -1,98 +0,0 @@ -#include -#include -#include "syncIO.h" -#include "syncInt.h" -#include "syncMessage.h" -#include "syncUtil.h" - -void logTest() { - sTrace("--- sync log test: trace"); - sDebug("--- sync log test: debug"); - sInfo("--- sync log test: info"); - sWarn("--- sync log test: warn"); - sError("--- sync log test: error"); - sFatal("--- sync log test: fatal"); -} - -SyncClientRequest *createMsg() { - SRpcMsg rpcMsg; - memset(&rpcMsg, 0, sizeof(rpcMsg)); - rpcMsg.msgType = 12345; - rpcMsg.contLen = 20; - rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); - strcpy((char *)rpcMsg.pCont, "hello rpc"); - SyncClientRequest *pMsg = syncClientRequestBuild(&rpcMsg, 123, true, 1000); - rpcFreeCont(rpcMsg.pCont); - return pMsg; -} - -void test1() { - SyncClientRequest *pMsg = createMsg(); - syncClientRequestLog2((char *)"test1:", pMsg); - syncClientRequestDestroy(pMsg); -} - -void test2() { - SyncClientRequest *pMsg = createMsg(); - uint32_t len = pMsg->bytes; - char *serialized = (char *)taosMemoryMalloc(len); - syncClientRequestSerialize(pMsg, serialized, len); - SyncClientRequest *pMsg2 = syncClientRequestAlloc(pMsg->dataLen); - syncClientRequestDeserialize(serialized, len, pMsg2); - syncClientRequestLog2((char *)"test2: syncClientRequestSerialize -> syncClientRequestDeserialize ", pMsg2); - - taosMemoryFree(serialized); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test3() { - SyncClientRequest *pMsg = createMsg(); - uint32_t len; - char *serialized = syncClientRequestSerialize2(pMsg, &len); - SyncClientRequest *pMsg2 = syncClientRequestDeserialize2(serialized, len); - syncClientRequestLog2((char *)"test3: syncClientRequestSerialize3 -> syncClientRequestDeserialize2 ", pMsg2); - - taosMemoryFree(serialized); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test4() { - SyncClientRequest *pMsg = createMsg(); - SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pMsg, &rpcMsg); - SyncClientRequest *pMsg2 = (SyncClientRequest *)taosMemoryMalloc(rpcMsg.contLen); - syncClientRequestFromRpcMsg(&rpcMsg, pMsg2); - syncClientRequestLog2((char *)"test4: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg ", pMsg2); - - rpcFreeCont(rpcMsg.pCont); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -void test5() { - SyncClientRequest *pMsg = createMsg(); - SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pMsg, &rpcMsg); - SyncClientRequest *pMsg2 = syncClientRequestFromRpcMsg2(&rpcMsg); - syncClientRequestLog2((char *)"test5: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg2 ", pMsg2); - - rpcFreeCont(rpcMsg.pCont); - syncClientRequestDestroy(pMsg); - syncClientRequestDestroy(pMsg2); -} - -int main() { - tsAsyncLog = 0; - sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; - logTest(); - - test1(); - test2(); - test3(); - test4(); - test5(); - - return 0; -} diff --git a/source/libs/sync/test/syncEncodeTest.cpp b/source/libs/sync/test/syncEncodeTest.cpp index c60176fbf82249721269107b1b7ab330c14fa0cf..216291c1d897a3eeaad8b13be503a81511e41b05 100644 --- a/source/libs/sync/test/syncEncodeTest.cpp +++ b/source/libs/sync/test/syncEncodeTest.cpp @@ -7,9 +7,11 @@ #include "syncRaftEntry.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "wal.h" +#if 0 void logTest() { sTrace("--- sync log test: trace"); sDebug("--- sync log test: debug"); @@ -118,17 +120,7 @@ SyncClientRequest *step3(const SRpcMsg *pMsg) { } SSyncRaftEntry *step4(const SyncClientRequest *pMsg) { - SSyncRaftEntry *pRetMsg = syncEntryBuild2((SyncClientRequest *)pMsg, 100, 0); - return pRetMsg; -} - -char *step5(const SSyncRaftEntry *pMsg, uint32_t *len) { - char *pRetMsg = syncEntrySerialize(pMsg, len); - return pRetMsg; -} - -SSyncRaftEntry *step6(const char *pMsg, uint32_t len) { - SSyncRaftEntry *pRetMsg = syncEntryDeserialize(pMsg, len); + SSyncRaftEntry *pRetMsg = syncEntryBuildFromClientRequest((SyncClientRequest *)pMsg, 100, 0); return pRetMsg; } @@ -137,13 +129,14 @@ SRpcMsg *step7(const SSyncRaftEntry *pMsg) { syncEntry2OriginalRpc(pMsg, pRetMsg); return pRetMsg; } - +#endif int main(int argc, char **argv) { // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; sDebugFlag = 143 + 64; void logTest(); +#if 0 myIndex = 0; if (argc >= 2) { myIndex = atoi(argv[1]); @@ -188,20 +181,9 @@ int main(int argc, char **argv) { syncEntryLog2((char *)"==pEntry==", pEntry); - // step5 - uint32_t len; - char *pMsg5 = step5(pMsg4, &len); - char *s = syncUtilPrintBin(pMsg5, len); - printf("==step5== [%s] \n", s); - taosMemoryFree(s); - - // step6 - SSyncRaftEntry *pMsg6 = step6(pMsg5, len); - syncEntryLog2((char *)"==step6==", pMsg6); - // step7 SRpcMsg *pMsg7 = step7(pMsg6); syncRpcMsgLog2((char *)"==step7==", pMsg7); - +#endif return 0; } diff --git a/source/libs/sync/test/syncEntryCacheTest.cpp b/source/libs/sync/test/syncEntryCacheTest.cpp index 3ee28ce96b00e0722402af586bdad30269af8e31..56b3d6da2e5bb7baea5253207c4dc52d9672b2db 100644 --- a/source/libs/sync/test/syncEntryCacheTest.cpp +++ b/source/libs/sync/test/syncEntryCacheTest.cpp @@ -7,6 +7,7 @@ #include "syncUtil.h" #include "tref.h" #include "tskiplist.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncEntryTest.cpp b/source/libs/sync/test/syncEntryTest.cpp index b274408c01024ed10de3ccb0b4bab08f48b7ffea..e94755195b97088a6d396ee08563454576c0c187 100644 --- a/source/libs/sync/test/syncEntryTest.cpp +++ b/source/libs/sync/test/syncEntryTest.cpp @@ -6,6 +6,7 @@ #include "syncRaftLog.h" #include "syncRaftStore.h" #include "syncUtil.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); @@ -38,10 +39,10 @@ void test2() { pSyncMsg->isWeak = 1; strcpy(pSyncMsg->data, "test2"); - SSyncRaftEntry* pEntry = syncEntryBuild2(pSyncMsg, 100, 200); + SSyncRaftEntry* pEntry = syncEntryBuildFromClientRequest(pSyncMsg, 100, 200); syncEntryPrint(pEntry); - syncClientRequestDestroy(pSyncMsg); + taosMemoryFree(pSyncMsg); syncEntryDestory(pEntry); } @@ -52,10 +53,10 @@ void test3() { pSyncMsg->isWeak = 1; strcpy(pSyncMsg->data, "test3"); - SSyncRaftEntry* pEntry = syncEntryBuild3(pSyncMsg, 100, 200); + SSyncRaftEntry* pEntry = syncEntryBuildFromClientRequest(pSyncMsg, 100, 200); syncEntryPrint(pEntry); - syncClientRequestDestroy(pSyncMsg); + taosMemoryFree(pSyncMsg); syncEntryDestory(pEntry); } @@ -71,14 +72,7 @@ void test4() { strcpy(pEntry->data, "test4"); syncEntryPrint(pEntry); - uint32_t len; - char* serialized = syncEntrySerialize(pEntry, &len); - assert(serialized != NULL); - SSyncRaftEntry* pEntry2 = syncEntryDeserialize(serialized, len); - syncEntryPrint(pEntry2); - - taosMemoryFree(serialized); - syncEntryDestory(pEntry2); + // syncEntryDestory(pEntry2); syncEntryDestory(pEntry); } diff --git a/source/libs/sync/test/syncHashCacheTest.cpp b/source/libs/sync/test/syncHashCacheTest.cpp index 7d822971da61462a7f9f65321bb2e72103fa3c27..2f5bb07f9a58fc8fe866eb177f5f009d1103c520 100644 --- a/source/libs/sync/test/syncHashCacheTest.cpp +++ b/source/libs/sync/test/syncHashCacheTest.cpp @@ -4,6 +4,7 @@ #include "syncInt.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "tskiplist.h" diff --git a/source/libs/sync/test/syncLogStoreCheck.cpp b/source/libs/sync/test/syncLogStoreCheck.cpp index 431b291ca797847a00556f5aeb6d231f48061dc6..0161160a75d5e2f5f9b116b10a2d4cbd7b64a36f 100644 --- a/source/libs/sync/test/syncLogStoreCheck.cpp +++ b/source/libs/sync/test/syncLogStoreCheck.cpp @@ -5,6 +5,7 @@ #include "syncInt.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "wal.h" diff --git a/source/libs/sync/test/syncLogStoreCheck2.cpp b/source/libs/sync/test/syncLogStoreCheck2.cpp index 80679bc85c4067fef291dcc583b727f1f3e909c3..29ad0610e7d8b23594cbd932a85bf60dd7488cf8 100644 --- a/source/libs/sync/test/syncLogStoreCheck2.cpp +++ b/source/libs/sync/test/syncLogStoreCheck2.cpp @@ -7,6 +7,7 @@ #include "syncRaftStore.h" #include "syncUtil.h" #include "wal.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncLogStoreTest.cpp b/source/libs/sync/test/syncLogStoreTest.cpp index 9ff0ed208957efc55ad66733d5a1543f44ef795e..832b42bf806280aac04b1116d27dc534012faae8 100644 --- a/source/libs/sync/test/syncLogStoreTest.cpp +++ b/source/libs/sync/test/syncLogStoreTest.cpp @@ -5,6 +5,7 @@ #include "syncInt.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "wal.h" diff --git a/source/libs/sync/test/syncRaftLogTest2.cpp b/source/libs/sync/test/syncRaftLogTest2.cpp index 78c08a6d8d9f0d832773f0f065dfffcbbf281825..3d50b63ff9feb3eaa1edb3b018023d911c456ef1 100644 --- a/source/libs/sync/test/syncRaftLogTest2.cpp +++ b/source/libs/sync/test/syncRaftLogTest2.cpp @@ -5,6 +5,7 @@ #include "syncInt.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "wal.h" diff --git a/source/libs/sync/test/syncRaftLogTest3.cpp b/source/libs/sync/test/syncRaftLogTest3.cpp index cf862feb2a9011b233eae6dc1bcbd2d7a3af4992..31c06625aa8eaa443a214b19bcac17463e63bda8 100644 --- a/source/libs/sync/test/syncRaftLogTest3.cpp +++ b/source/libs/sync/test/syncRaftLogTest3.cpp @@ -1,12 +1,5 @@ #include -#include -#include "syncEnv.h" -#include "syncIO.h" -#include "syncInt.h" -#include "syncRaftLog.h" -#include "syncRaftStore.h" -#include "syncUtil.h" -#include "wal.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncRespMgrTest.cpp b/source/libs/sync/test/syncRespMgrTest.cpp index cad6eec91d204e74d0b0534f6cc9de13df3f0dca..8d709e8c81b5d84cf1cd9e414940ece1ce9f5823 100644 --- a/source/libs/sync/test/syncRespMgrTest.cpp +++ b/source/libs/sync/test/syncRespMgrTest.cpp @@ -1,8 +1,4 @@ -#include "syncRespMgr.h" -//#include -#include -#include "syncIO.h" -#include "syncInt.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); @@ -64,13 +60,13 @@ void syncRespMgrGetTest(uint64_t i) { void syncRespMgrGetAndDelTest(uint64_t i) { printf("------syncRespMgrGetAndDelTest-------%" PRIu64 "-- \n", i); - SRespStub stub; - int32_t ret = syncRespMgrGetAndDel(pMgr, i, &stub); - if (ret == 1) { - printStub(&stub); - } else if (ret == 0) { - printf("%" PRId64 " notFound \n", i); - } + // SRespStub stub; + // int32_t ret = syncRespMgrGetAndDel(pMgr, i, &stub); + // if (ret == 1) { + // printStub(&stub); + // } else if (ret == 0) { + // printf("%" PRId64 " notFound \n", i); + // } } SSyncNode *createSyncNode() { diff --git a/source/libs/sync/test/syncRpcMsgTest.cpp b/source/libs/sync/test/syncRpcMsgTest.cpp index 941fa7eab5544576170a59bd6f73b122e32d78cb..127d8e1c4148caa0d7e12f3cc13b70a7e4ea3d22 100644 --- a/source/libs/sync/test/syncRpcMsgTest.cpp +++ b/source/libs/sync/test/syncRpcMsgTest.cpp @@ -47,7 +47,11 @@ SyncClientRequest *createSyncClientRequest() { rpcMsg.contLen = 20; rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); strcpy((char *)rpcMsg.pCont, "hello rpc"); - SyncClientRequest *pMsg = syncClientRequestBuild(&rpcMsg, 123, true, 1000); + + SRpcMsg clientRequestMsg; + syncClientRequestBuildFromRpcMsg(&clientRequestMsg, &rpcMsg, 123, true, 1000); + SyncClientRequest *pMsg = (SyncClientRequest *)taosMemoryMalloc(clientRequestMsg.contLen); + memcpy(pMsg->data, clientRequestMsg.pCont, clientRequestMsg.contLen); return pMsg; } @@ -155,11 +159,13 @@ void test7() { } void test8() { +#if 0 SyncClientRequest *pMsg = createSyncClientRequest(); SRpcMsg rpcMsg = {0}; syncClientRequest2RpcMsg(pMsg, &rpcMsg); syncRpcMsgLog2((char *)"test8", &rpcMsg); - syncClientRequestDestroy(pMsg); + taosMemoryFree(pMsg); +#endif } int main() { diff --git a/source/libs/sync/test/syncSnapshotTest.cpp b/source/libs/sync/test/syncSnapshotTest.cpp index a1cedd624a144f7437bad235818dc515f72047fc..e2264fd08c4cd7bf80981aee66eec4a0377aa0b3 100644 --- a/source/libs/sync/test/syncSnapshotTest.cpp +++ b/source/libs/sync/test/syncSnapshotTest.cpp @@ -162,8 +162,11 @@ SRpcMsg *step0() { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild(pMsg, 123, true, 1000); - return pRetMsg; + SRpcMsg clientRequestMsg; + syncClientRequestBuildFromRpcMsg(&clientRequestMsg, pMsg, 123, true, 1000); + SyncClientRequest *pMsg2 = (SyncClientRequest *)taosMemoryMalloc(clientRequestMsg.contLen); + memcpy(pMsg2->data, clientRequestMsg.pCont, clientRequestMsg.contLen); + return pMsg2; } int main(int argc, char **argv) { @@ -207,8 +210,8 @@ int main(int argc, char **argv) { for (int i = 0; i < 10; ++i) { SyncClientRequest *pSyncClientRequest = pMsg1; SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pSyncClientRequest, &rpcMsg); - gSyncNode->syncEqMsg(gSyncNode->msgcb, &rpcMsg); + // syncClientRequest2RpcMsg(pSyncClientRequest, &rpcMsg); + // gSyncNode->syncEqMsg(gSyncNode->msgcb, &rpcMsg); taosMsleep(1000); } diff --git a/source/libs/sync/test/syncVotesGrantedTest.cpp b/source/libs/sync/test/syncVotesGrantedTest.cpp index bbf14c604ec155d078b414fd24f1240940ab9a74..6a8404308b4985ab026c22499b68e39c7601a034 100644 --- a/source/libs/sync/test/syncVotesGrantedTest.cpp +++ b/source/libs/sync/test/syncVotesGrantedTest.cpp @@ -6,6 +6,7 @@ #include "syncRaftStore.h" #include "syncUtil.h" #include "syncVoteMgr.h" +#include "syncTest.h" void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncVotesRespondTest.cpp b/source/libs/sync/test/syncVotesRespondTest.cpp index adebfe1be2645b19e7e846c199cbacd5444595fa..d5e7d71030d0346b2d32cb1e1df4b966a4833fb7 100644 --- a/source/libs/sync/test/syncVotesRespondTest.cpp +++ b/source/libs/sync/test/syncVotesRespondTest.cpp @@ -6,6 +6,8 @@ #include "syncRaftStore.h" #include "syncUtil.h" #include "syncVoteMgr.h" +#include "syncTest.h" + void logTest() { sTrace("--- sync log test: trace"); diff --git a/source/libs/sync/test/syncWriteTest.cpp b/source/libs/sync/test/syncWriteTest.cpp index 7c5334f668a690e33625772cdbdb56a9d873b037..2e5c26719ba39ab67fa1408937e9082ec3a9d5fa 100644 --- a/source/libs/sync/test/syncWriteTest.cpp +++ b/source/libs/sync/test/syncWriteTest.cpp @@ -7,6 +7,7 @@ #include "syncRaftEntry.h" #include "syncRaftLog.h" #include "syncRaftStore.h" +#include "syncTest.h" #include "syncUtil.h" #include "wal.h" @@ -140,7 +141,8 @@ SRpcMsg *step0() { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild(pMsg, 123, true, 1000); + SyncClientRequest *pRetMsg = NULL; + // syncClientRequestBuild(pMsg, 123, true, 1000); return pRetMsg; } @@ -179,14 +181,14 @@ int main(int argc, char **argv) { SyncClientRequest *pMsg1 = step1(pMsg0); syncClientRequestLog2((char *)"==step1==", pMsg1); - for (int i = 0; i < 10; ++i) { - SyncClientRequest *pSyncClientRequest = pMsg1; - SRpcMsg rpcMsg = {0}; - syncClientRequest2RpcMsg(pSyncClientRequest, &rpcMsg); - gSyncNode->syncEqMsg(gSyncNode->msgcb, &rpcMsg); + // for (int i = 0; i < 10; ++i) { + // SyncClientRequest *pSyncClientRequest = pMsg1; + // SRpcMsg rpcMsg = {0}; + // syncClientRequest2RpcMsg(pSyncClientRequest, &rpcMsg); + // gSyncNode->syncEqMsg(gSyncNode->msgcb, &rpcMsg); - taosMsleep(1000); - } + // taosMsleep(1000); + // } while (1) { sTrace("while 1 sleep"); diff --git a/source/libs/sync/test/sync_test_lib/CMakeLists.txt b/source/libs/sync/test/sync_test_lib/CMakeLists.txt new file mode 100644 index 0000000000000000000000000000000000000000..db775c3662025be3f94ef33f32db0da28ec0cd60 --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/CMakeLists.txt @@ -0,0 +1,14 @@ +aux_source_directory(src SYNC_TEST_SRC) +add_library(sync_test_lib STATIC ${SYNC_TEST_SRC}) + +target_link_libraries( + sync_test_lib + PUBLIC sync +) + +target_include_directories( + sync_test_lib + PUBLIC "${TD_SOURCE_DIR}/include/libs/sync" + PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../inc" + PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/inc" +) diff --git a/source/libs/sync/inc/syncIO.h b/source/libs/sync/test/sync_test_lib/inc/syncIO.h similarity index 95% rename from source/libs/sync/inc/syncIO.h rename to source/libs/sync/test/sync_test_lib/inc/syncIO.h index cfc4dd247263a3c3eb877a814d6da00ed956589a..955a832b6862321b723e422f6969be8bc798f653 100644 --- a/source/libs/sync/inc/syncIO.h +++ b/source/libs/sync/test/sync_test_lib/inc/syncIO.h @@ -50,7 +50,7 @@ typedef struct SSyncIO { void *pSyncNode; int32_t (*FpOnSyncPing)(SSyncNode *pSyncNode, SyncPing *pMsg); int32_t (*FpOnSyncPingReply)(SSyncNode *pSyncNode, SyncPingReply *pMsg); - int32_t (*FpOnSyncClientRequest)(SSyncNode *pSyncNode, SyncClientRequest *pMsg, SyncIndex *pRetIndex); + int32_t (*FpOnSyncClientRequest)(SSyncNode *pSyncNode, SRpcMsg *pMsg, SyncIndex *pRetIndex); int32_t (*FpOnSyncRequestVote)(SSyncNode *pSyncNode, SyncRequestVote *pMsg); int32_t (*FpOnSyncRequestVoteReply)(SSyncNode *pSyncNode, SyncRequestVoteReply *pMsg); int32_t (*FpOnSyncAppendEntries)(SSyncNode *pSyncNode, SyncAppendEntries *pMsg); diff --git a/source/libs/sync/test/sync_test_lib/inc/syncTest.h b/source/libs/sync/test/sync_test_lib/inc/syncTest.h new file mode 100644 index 0000000000000000000000000000000000000000..3a490742724c51387676ec29a485fce0be90f84f --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/inc/syncTest.h @@ -0,0 +1,90 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef _TD_LIBS_SYNC_TEST_H +#define _TD_LIBS_SYNC_TEST_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include "syncInt.h" + +#include "wal.h" + +#include "syncEnv.h" +#include "syncIO.h" +#include "syncIndexMgr.h" +#include "syncInt.h" +#include "syncMessage.h" +#include "syncRaftCfg.h" +#include "syncRaftEntry.h" +#include "syncRaftLog.h" +#include "syncRaftStore.h" +#include "syncRespMgr.h" +#include "syncSnapshot.h" +#include "syncUtil.h" +#include "syncVoteMgr.h" + +cJSON* syncEntry2Json(const SSyncRaftEntry* pEntry); +char* syncEntry2Str(const SSyncRaftEntry* pEntry); +void syncEntryPrint(const SSyncRaftEntry* pObj); +void syncEntryPrint2(char* s, const SSyncRaftEntry* pObj); +void syncEntryLog(const SSyncRaftEntry* pObj); +void syncEntryLog2(char* s, const SSyncRaftEntry* pObj); + +cJSON* raftCache2Json(SRaftEntryHashCache* pObj); +char* raftCache2Str(SRaftEntryHashCache* pObj); +void raftCachePrint(SRaftEntryHashCache* pObj); +void raftCachePrint2(char* s, SRaftEntryHashCache* pObj); +void raftCacheLog(SRaftEntryHashCache* pObj); +void raftCacheLog2(char* s, SRaftEntryHashCache* pObj); + +cJSON* raftEntryCache2Json(SRaftEntryCache* pObj); +char* raftEntryCache2Str(SRaftEntryCache* pObj); +void raftEntryCachePrint(SRaftEntryCache* pObj); +void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj); +void raftEntryCacheLog(SRaftEntryCache* pObj); +void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj); + +cJSON* syncAppendEntriesBatch2Json(const SyncAppendEntriesBatch* pMsg); +char* syncAppendEntriesBatch2Str(const SyncAppendEntriesBatch* pMsg); +void syncAppendEntriesBatchPrint(const SyncAppendEntriesBatch* pMsg); +void syncAppendEntriesBatchPrint2(char* s, const SyncAppendEntriesBatch* pMsg); +void syncAppendEntriesBatchLog(const SyncAppendEntriesBatch* pMsg); +void syncAppendEntriesBatchLog2(char* s, const SyncAppendEntriesBatch* pMsg); + +cJSON* logStore2Json(SSyncLogStore* pLogStore); +char* logStore2Str(SSyncLogStore* pLogStore); +cJSON* logStoreSimple2Json(SSyncLogStore* pLogStore); +char* logStoreSimple2Str(SSyncLogStore* pLogStore); +void logStorePrint(SSyncLogStore* pLogStore); +void logStorePrint2(char* s, SSyncLogStore* pLogStore); +void logStoreLog(SSyncLogStore* pLogStore); +void logStoreLog2(char* s, SSyncLogStore* pLogStore); +void logStoreSimplePrint(SSyncLogStore* pLogStore); +void logStoreSimplePrint2(char* s, SSyncLogStore* pLogStore); +void logStoreSimpleLog(SSyncLogStore* pLogStore); +void logStoreSimpleLog2(char* s, SSyncLogStore* pLogStore); + +cJSON* syncNode2Json(const SSyncNode* pSyncNode); +char* syncNode2Str(const SSyncNode* pSyncNode); + + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_LIBS_SYNC_RAFT_ENTRY_H*/ diff --git a/source/libs/sync/src/syncIO.c b/source/libs/sync/test/sync_test_lib/src/syncIO.c similarity index 98% rename from source/libs/sync/src/syncIO.c rename to source/libs/sync/test/sync_test_lib/src/syncIO.c index afa2d43e13a90000da15093b658ca480b285791f..14adc18c66f3255a49aee91376e767e610735ae4 100644 --- a/source/libs/sync/src/syncIO.c +++ b/source/libs/sync/test/sync_test_lib/src/syncIO.c @@ -279,10 +279,7 @@ static void *syncIOConsumerFunc(void *param) { } else if (pRpcMsg->msgType == TDMT_SYNC_CLIENT_REQUEST) { if (io->FpOnSyncClientRequest != NULL) { - SyncClientRequest *pSyncMsg = syncClientRequestFromRpcMsg2(pRpcMsg); - ASSERT(pSyncMsg != NULL); - io->FpOnSyncClientRequest(io->pSyncNode, pSyncMsg, NULL); - syncClientRequestDestroy(pSyncMsg); + io->FpOnSyncClientRequest(io->pSyncNode, pRpcMsg, NULL); } } else if (pRpcMsg->msgType == TDMT_SYNC_REQUEST_VOTE) { diff --git a/source/libs/sync/test/sync_test_lib/src/syncMainDebug.c b/source/libs/sync/test/sync_test_lib/src/syncMainDebug.c new file mode 100644 index 0000000000000000000000000000000000000000..99d124580bf46351e5d396e17a879d781e77c049 --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/src/syncMainDebug.c @@ -0,0 +1,255 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "syncTest.h" + +cJSON* syncNode2Json(const SSyncNode* pSyncNode) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pSyncNode != NULL) { + // init by SSyncInfo + cJSON_AddNumberToObject(pRoot, "vgId", pSyncNode->vgId); + cJSON_AddItemToObject(pRoot, "SRaftCfg", raftCfg2Json(pSyncNode->pRaftCfg)); + cJSON_AddStringToObject(pRoot, "path", pSyncNode->path); + cJSON_AddStringToObject(pRoot, "raftStorePath", pSyncNode->raftStorePath); + cJSON_AddStringToObject(pRoot, "configPath", pSyncNode->configPath); + + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pWal); + cJSON_AddStringToObject(pRoot, "pWal", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->msgcb); + cJSON_AddStringToObject(pRoot, "rpcClient", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->syncSendMSg); + cJSON_AddStringToObject(pRoot, "syncSendMSg", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->msgcb); + cJSON_AddStringToObject(pRoot, "queue", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->syncEqMsg); + cJSON_AddStringToObject(pRoot, "syncEqMsg", u64buf); + + // init internal + cJSON* pMe = syncUtilNodeInfo2Json(&pSyncNode->myNodeInfo); + cJSON_AddItemToObject(pRoot, "myNodeInfo", pMe); + cJSON* pRaftId = syncUtilRaftId2Json(&pSyncNode->myRaftId); + cJSON_AddItemToObject(pRoot, "myRaftId", pRaftId); + + cJSON_AddNumberToObject(pRoot, "peersNum", pSyncNode->peersNum); + cJSON* pPeers = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "peersNodeInfo", pPeers); + for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { + cJSON_AddItemToArray(pPeers, syncUtilNodeInfo2Json(&pSyncNode->peersNodeInfo[i])); + } + cJSON* pPeersId = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "peersId", pPeersId); + for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { + cJSON_AddItemToArray(pPeersId, syncUtilRaftId2Json(&pSyncNode->peersId[i])); + } + + cJSON_AddNumberToObject(pRoot, "replicaNum", pSyncNode->replicaNum); + cJSON* pReplicasId = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "replicasId", pReplicasId); + for (int32_t i = 0; i < pSyncNode->replicaNum; ++i) { + cJSON_AddItemToArray(pReplicasId, syncUtilRaftId2Json(&pSyncNode->replicasId[i])); + } + + // raft algorithm + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pFsm); + cJSON_AddStringToObject(pRoot, "pFsm", u64buf); + cJSON_AddNumberToObject(pRoot, "quorum", pSyncNode->quorum); + cJSON* pLaderCache = syncUtilRaftId2Json(&pSyncNode->leaderCache); + cJSON_AddItemToObject(pRoot, "leaderCache", pLaderCache); + + // life cycle + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pSyncNode->rid); + cJSON_AddStringToObject(pRoot, "rid", u64buf); + + // tla+ server vars + cJSON_AddNumberToObject(pRoot, "state", pSyncNode->state); + cJSON_AddStringToObject(pRoot, "state_str", syncStr(pSyncNode->state)); + cJSON_AddItemToObject(pRoot, "pRaftStore", raftStore2Json(pSyncNode->pRaftStore)); + + // tla+ candidate vars + cJSON_AddItemToObject(pRoot, "pVotesGranted", voteGranted2Json(pSyncNode->pVotesGranted)); + cJSON_AddItemToObject(pRoot, "pVotesRespond", votesRespond2Json(pSyncNode->pVotesRespond)); + + // tla+ leader vars + cJSON_AddItemToObject(pRoot, "pNextIndex", syncIndexMgr2Json(pSyncNode->pNextIndex)); + cJSON_AddItemToObject(pRoot, "pMatchIndex", syncIndexMgr2Json(pSyncNode->pMatchIndex)); + + // tla+ log vars + cJSON_AddItemToObject(pRoot, "pLogStore", logStore2Json(pSyncNode->pLogStore)); + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pSyncNode->commitIndex); + cJSON_AddStringToObject(pRoot, "commitIndex", u64buf); + + // timer ms init + cJSON_AddNumberToObject(pRoot, "pingBaseLine", pSyncNode->pingBaseLine); + cJSON_AddNumberToObject(pRoot, "electBaseLine", pSyncNode->electBaseLine); + cJSON_AddNumberToObject(pRoot, "hbBaseLine", pSyncNode->hbBaseLine); + + // ping timer + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pPingTimer); + cJSON_AddStringToObject(pRoot, "pPingTimer", u64buf); + cJSON_AddNumberToObject(pRoot, "pingTimerMS", pSyncNode->pingTimerMS); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerLogicClock); + cJSON_AddStringToObject(pRoot, "pingTimerLogicClock", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerLogicClockUser); + cJSON_AddStringToObject(pRoot, "pingTimerLogicClockUser", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpPingTimerCB); + cJSON_AddStringToObject(pRoot, "FpPingTimerCB", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->pingTimerCounter); + cJSON_AddStringToObject(pRoot, "pingTimerCounter", u64buf); + + // elect timer + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pElectTimer); + cJSON_AddStringToObject(pRoot, "pElectTimer", u64buf); + cJSON_AddNumberToObject(pRoot, "electTimerMS", pSyncNode->electTimerMS); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->electTimerLogicClock); + cJSON_AddStringToObject(pRoot, "electTimerLogicClock", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpElectTimerCB); + cJSON_AddStringToObject(pRoot, "FpElectTimerCB", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->electTimerCounter); + cJSON_AddStringToObject(pRoot, "electTimerCounter", u64buf); + + // heartbeat timer + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->pHeartbeatTimer); + cJSON_AddStringToObject(pRoot, "pHeartbeatTimer", u64buf); + cJSON_AddNumberToObject(pRoot, "heartbeatTimerMS", pSyncNode->heartbeatTimerMS); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerLogicClock); + cJSON_AddStringToObject(pRoot, "heartbeatTimerLogicClock", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerLogicClockUser); + cJSON_AddStringToObject(pRoot, "heartbeatTimerLogicClockUser", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpHeartbeatTimerCB); + cJSON_AddStringToObject(pRoot, "FpHeartbeatTimerCB", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pSyncNode->heartbeatTimerCounter); + cJSON_AddStringToObject(pRoot, "heartbeatTimerCounter", u64buf); + + // callback + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnPing); + cJSON_AddStringToObject(pRoot, "FpOnPing", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnPingReply); + cJSON_AddStringToObject(pRoot, "FpOnPingReply", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnRequestVote); + cJSON_AddStringToObject(pRoot, "FpOnRequestVote", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnRequestVoteReply); + cJSON_AddStringToObject(pRoot, "FpOnRequestVoteReply", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnAppendEntries); + cJSON_AddStringToObject(pRoot, "FpOnAppendEntries", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnAppendEntriesReply); + cJSON_AddStringToObject(pRoot, "FpOnAppendEntriesReply", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pSyncNode->FpOnTimeout); + cJSON_AddStringToObject(pRoot, "FpOnTimeout", u64buf); + + // restoreFinish + cJSON_AddNumberToObject(pRoot, "restoreFinish", pSyncNode->restoreFinish); + + // snapshot senders + cJSON* pSenders = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "senders", pSenders); + for (int32_t i = 0; i < TSDB_MAX_REPLICA; ++i) { + cJSON_AddItemToArray(pSenders, snapshotSender2Json((pSyncNode->senders)[i])); + } + + // snapshot receivers + cJSON* pReceivers = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "receiver", snapshotReceiver2Json(pSyncNode->pNewNodeReceiver)); + + // changing + cJSON_AddNumberToObject(pRoot, "changing", pSyncNode->changing); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SSyncNode", pRoot); + return pJson; +} + +char* syncNode2Str(const SSyncNode* pSyncNode) { + cJSON* pJson = syncNode2Json(pSyncNode); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +inline char* syncNode2SimpleStr(const SSyncNode* pSyncNode) { + int32_t len = 256; + char* s = (char*)taosMemoryMalloc(len); + + SSnapshot snapshot = {.data = NULL, .lastApplyIndex = -1, .lastApplyTerm = 0}; + if (pSyncNode->pFsm->FpGetSnapshotInfo != NULL) { + pSyncNode->pFsm->FpGetSnapshotInfo(pSyncNode->pFsm, &snapshot); + } + SyncIndex logLastIndex = pSyncNode->pLogStore->syncLogLastIndex(pSyncNode->pLogStore); + SyncIndex logBeginIndex = pSyncNode->pLogStore->syncLogBeginIndex(pSyncNode->pLogStore); + + snprintf(s, len, + "vgId:%d, sync %s, tm:%" PRIu64 ", cmt:%" PRId64 ", fst:%" PRId64 ", lst:%" PRId64 ", snap:%" PRId64 + ", sby:%d, " + "r-num:%d, " + "lcfg:%" PRId64 ", chging:%d, rsto:%d", + pSyncNode->vgId, syncStr(pSyncNode->state), pSyncNode->pRaftStore->currentTerm, pSyncNode->commitIndex, + logBeginIndex, logLastIndex, snapshot.lastApplyIndex, pSyncNode->pRaftCfg->isStandBy, pSyncNode->replicaNum, + pSyncNode->pRaftCfg->lastConfigIndex, pSyncNode->changing, pSyncNode->restoreFinish); + + return s; +} + +// ping -------------- +int32_t syncNodePing(SSyncNode* pSyncNode, const SRaftId* destRaftId, SyncPing* pMsg) { + syncPingLog2((char*)"==syncNodePing==", pMsg); + int32_t ret = 0; + + SRpcMsg rpcMsg; + syncPing2RpcMsg(pMsg, &rpcMsg); + syncRpcMsgLog2((char*)"==syncNodePing==", &rpcMsg); + + ret = syncNodeSendMsgById(destRaftId, pSyncNode, &rpcMsg); + return ret; +} + +int32_t syncNodePingSelf(SSyncNode* pSyncNode) { + int32_t ret = 0; + SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &pSyncNode->myRaftId, pSyncNode->vgId); + ret = syncNodePing(pSyncNode, &pMsg->destId, pMsg); + ASSERT(ret == 0); + + syncPingDestroy(pMsg); + return ret; +} + +int32_t syncNodePingPeers(SSyncNode* pSyncNode) { + int32_t ret = 0; + for (int32_t i = 0; i < pSyncNode->peersNum; ++i) { + SRaftId* destId = &(pSyncNode->peersId[i]); + SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, destId, pSyncNode->vgId); + ret = syncNodePing(pSyncNode, destId, pMsg); + ASSERT(ret == 0); + syncPingDestroy(pMsg); + } + return ret; +} + +int32_t syncNodePingAll(SSyncNode* pSyncNode) { + int32_t ret = 0; + for (int32_t i = 0; i < pSyncNode->pRaftCfg->cfg.replicaNum; ++i) { + SRaftId* destId = &(pSyncNode->replicasId[i]); + SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, destId, pSyncNode->vgId); + ret = syncNodePing(pSyncNode, destId, pMsg); + ASSERT(ret == 0); + syncPingDestroy(pMsg); + } + return ret; +} + diff --git a/source/libs/sync/test/sync_test_lib/src/syncMessageDebug.c b/source/libs/sync/test/sync_test_lib/src/syncMessageDebug.c new file mode 100644 index 0000000000000000000000000000000000000000..012382d69d3bcda0a4f6cd2c2daae6f9b7830a99 --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/src/syncMessageDebug.c @@ -0,0 +1,508 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "syncTest.h" + +// --------------------------------------------- +cJSON* syncRpcMsg2Json(SRpcMsg* pRpcMsg) { + cJSON* pRoot; + + // in compiler optimization, switch case = if else constants + if (pRpcMsg->msgType == TDMT_SYNC_TIMEOUT) { + SyncTimeout* pSyncMsg = syncTimeoutDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncTimeout2Json(pSyncMsg); + syncTimeoutDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_PING) { + SyncPing* pSyncMsg = syncPingDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncPing2Json(pSyncMsg); + syncPingDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_PING_REPLY) { + SyncPingReply* pSyncMsg = syncPingReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncPingReply2Json(pSyncMsg); + syncPingReplyDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_CLIENT_REQUEST) { + SyncClientRequest* pSyncMsg = pRpcMsg->pCont; + pRoot = syncClientRequest2Json(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_CLIENT_REQUEST_REPLY) { + pRoot = syncRpcUnknownMsg2Json(); + + } else if (pRpcMsg->msgType == TDMT_SYNC_REQUEST_VOTE) { + SyncRequestVote* pSyncMsg = syncRequestVoteDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncRequestVote2Json(pSyncMsg); + syncRequestVoteDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_REQUEST_VOTE_REPLY) { + SyncRequestVoteReply* pSyncMsg = syncRequestVoteReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncRequestVoteReply2Json(pSyncMsg); + syncRequestVoteReplyDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_APPEND_ENTRIES) { + SyncAppendEntries* pSyncMsg = syncAppendEntriesDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncAppendEntries2Json(pSyncMsg); + syncAppendEntriesDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_APPEND_ENTRIES_REPLY) { + SyncAppendEntriesReply* pSyncMsg = syncAppendEntriesReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncAppendEntriesReply2Json(pSyncMsg); + syncAppendEntriesReplyDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_SNAPSHOT_SEND) { + SyncSnapshotSend* pSyncMsg = syncSnapshotSendDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncSnapshotSend2Json(pSyncMsg); + syncSnapshotSendDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_SNAPSHOT_RSP) { + SyncSnapshotRsp* pSyncMsg = syncSnapshotRspDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncSnapshotRsp2Json(pSyncMsg); + syncSnapshotRspDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_LEADER_TRANSFER) { + SyncLeaderTransfer* pSyncMsg = syncLeaderTransferDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); + pRoot = syncLeaderTransfer2Json(pSyncMsg); + syncLeaderTransferDestroy(pSyncMsg); + + } else if (pRpcMsg->msgType == TDMT_SYNC_COMMON_RESPONSE) { + pRoot = cJSON_CreateObject(); + char* s; + s = syncUtilPrintBin((char*)(pRpcMsg->pCont), pRpcMsg->contLen); + cJSON_AddStringToObject(pRoot, "pCont", s); + taosMemoryFree(s); + s = syncUtilPrintBin2((char*)(pRpcMsg->pCont), pRpcMsg->contLen); + cJSON_AddStringToObject(pRoot, "pCont2", s); + taosMemoryFree(s); + + } else { + pRoot = cJSON_CreateObject(); + char* s; + s = syncUtilPrintBin((char*)(pRpcMsg->pCont), pRpcMsg->contLen); + cJSON_AddStringToObject(pRoot, "pCont", s); + taosMemoryFree(s); + s = syncUtilPrintBin2((char*)(pRpcMsg->pCont), pRpcMsg->contLen); + cJSON_AddStringToObject(pRoot, "pCont2", s); + taosMemoryFree(s); + } + + cJSON_AddNumberToObject(pRoot, "msgType", pRpcMsg->msgType); + cJSON_AddNumberToObject(pRoot, "contLen", pRpcMsg->contLen); + cJSON_AddNumberToObject(pRoot, "code", pRpcMsg->code); + // cJSON_AddNumberToObject(pRoot, "persist", pRpcMsg->persist); + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "RpcMsg", pRoot); + return pJson; +} + +cJSON* syncRpcUnknownMsg2Json() { + cJSON* pRoot = cJSON_CreateObject(); + cJSON_AddNumberToObject(pRoot, "msgType", TDMT_SYNC_UNKNOWN); + cJSON_AddStringToObject(pRoot, "data", "unknown message"); + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SyncUnknown", pRoot); + return pJson; +} + +char* syncRpcMsg2Str(SRpcMsg* pRpcMsg) { + cJSON* pJson = syncRpcMsg2Json(pRpcMsg); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ---------------------- +void syncRpcMsgPrint(SRpcMsg* pMsg) { + char* serialized = syncRpcMsg2Str(pMsg); + printf("syncRpcMsgPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncRpcMsgPrint2(char* s, SRpcMsg* pMsg) { + char* serialized = syncRpcMsg2Str(pMsg); + printf("syncRpcMsgPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncRpcMsgLog(SRpcMsg* pMsg) { + char* serialized = syncRpcMsg2Str(pMsg); + sTrace("syncRpcMsgLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncRpcMsgLog2(char* s, SRpcMsg* pMsg) { + if (gRaftDetailLog) { + char* serialized = syncRpcMsg2Str(pMsg); + sTrace("syncRpcMsgLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +cJSON* syncAppendEntriesBatch2Json(const SyncAppendEntriesBatch* pMsg) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pMsg != NULL) { + cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); + cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); + cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); + + cJSON* pSrcId = cJSON_CreateObject(); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->srcId.addr); + cJSON_AddStringToObject(pSrcId, "addr", u64buf); + { + uint64_t u64 = pMsg->srcId.addr; + cJSON* pTmp = pSrcId; + char host[128] = {0}; + uint16_t port; + syncUtilU642Addr(u64, host, sizeof(host), &port); + cJSON_AddStringToObject(pTmp, "addr_host", host); + cJSON_AddNumberToObject(pTmp, "addr_port", port); + } + cJSON_AddNumberToObject(pSrcId, "vgId", pMsg->srcId.vgId); + cJSON_AddItemToObject(pRoot, "srcId", pSrcId); + + cJSON* pDestId = cJSON_CreateObject(); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->destId.addr); + cJSON_AddStringToObject(pDestId, "addr", u64buf); + { + uint64_t u64 = pMsg->destId.addr; + cJSON* pTmp = pDestId; + char host[128] = {0}; + uint16_t port; + syncUtilU642Addr(u64, host, sizeof(host), &port); + cJSON_AddStringToObject(pTmp, "addr_host", host); + cJSON_AddNumberToObject(pTmp, "addr_port", port); + } + cJSON_AddNumberToObject(pDestId, "vgId", pMsg->destId.vgId); + cJSON_AddItemToObject(pRoot, "destId", pDestId); + + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term); + cJSON_AddStringToObject(pRoot, "term", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->prevLogIndex); + cJSON_AddStringToObject(pRoot, "prevLogIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->prevLogTerm); + cJSON_AddStringToObject(pRoot, "prevLogTerm", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->commitIndex); + cJSON_AddStringToObject(pRoot, "commitIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->privateTerm); + cJSON_AddStringToObject(pRoot, "privateTerm", u64buf); + + cJSON_AddNumberToObject(pRoot, "dataCount", pMsg->dataCount); + cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); + + int32_t metaArrayLen = sizeof(SOffsetAndContLen) * pMsg->dataCount; // + int32_t entryArrayLen = pMsg->dataLen - metaArrayLen; + + cJSON_AddNumberToObject(pRoot, "metaArrayLen", metaArrayLen); + cJSON_AddNumberToObject(pRoot, "entryArrayLen", entryArrayLen); + + SOffsetAndContLen* metaArr = (SOffsetAndContLen*)(pMsg->data); + + cJSON* pMetaArr = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "metaArr", pMetaArr); + for (int i = 0; i < pMsg->dataCount; ++i) { + cJSON* pMeta = cJSON_CreateObject(); + cJSON_AddNumberToObject(pMeta, "offset", metaArr[i].offset); + cJSON_AddNumberToObject(pMeta, "contLen", metaArr[i].contLen); + cJSON_AddItemToArray(pMetaArr, pMeta); + } + + cJSON* pEntryArr = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "entryArr", pEntryArr); + for (int i = 0; i < pMsg->dataCount; ++i) { + SSyncRaftEntry* pEntry = (SSyncRaftEntry*)(pMsg->data + metaArr[i].offset); + cJSON* pEntryJson = syncEntry2Json(pEntry); + cJSON_AddItemToArray(pEntryArr, pEntryJson); + } + + char* s; + s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data", s); + taosMemoryFree(s); + s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data2", s); + taosMemoryFree(s); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SyncAppendEntriesBatch", pRoot); + return pJson; +} + +char* syncAppendEntriesBatch2Str(const SyncAppendEntriesBatch* pMsg) { + cJSON* pJson = syncAppendEntriesBatch2Json(pMsg); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ---------------------- +void syncAppendEntriesBatchPrint(const SyncAppendEntriesBatch* pMsg) { + char* serialized = syncAppendEntriesBatch2Str(pMsg); + printf("syncAppendEntriesBatchPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncAppendEntriesBatchPrint2(char* s, const SyncAppendEntriesBatch* pMsg) { + char* serialized = syncAppendEntriesBatch2Str(pMsg); + printf("syncAppendEntriesBatchPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncAppendEntriesBatchLog(const SyncAppendEntriesBatch* pMsg) { + char* serialized = syncAppendEntriesBatch2Str(pMsg); + sTrace("syncAppendEntriesBatchLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncAppendEntriesBatchLog2(char* s, const SyncAppendEntriesBatch* pMsg) { + if (gRaftDetailLog) { + char* serialized = syncAppendEntriesBatch2Str(pMsg); + sLTrace("syncAppendEntriesBatchLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +cJSON* syncSnapshotSend2Json(const SyncSnapshotSend* pMsg) { + char u64buf[128]; + cJSON* pRoot = cJSON_CreateObject(); + + if (pMsg != NULL) { + cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); + cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); + cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); + + cJSON* pSrcId = cJSON_CreateObject(); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->srcId.addr); + cJSON_AddStringToObject(pSrcId, "addr", u64buf); + { + uint64_t u64 = pMsg->srcId.addr; + cJSON* pTmp = pSrcId; + char host[128]; + uint16_t port; + syncUtilU642Addr(u64, host, sizeof(host), &port); + cJSON_AddStringToObject(pTmp, "addr_host", host); + cJSON_AddNumberToObject(pTmp, "addr_port", port); + } + cJSON_AddNumberToObject(pSrcId, "vgId", pMsg->srcId.vgId); + cJSON_AddItemToObject(pRoot, "srcId", pSrcId); + + cJSON* pDestId = cJSON_CreateObject(); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->destId.addr); + cJSON_AddStringToObject(pDestId, "addr", u64buf); + { + uint64_t u64 = pMsg->destId.addr; + cJSON* pTmp = pDestId; + char host[128]; + uint16_t port; + syncUtilU642Addr(u64, host, sizeof(host), &port); + cJSON_AddStringToObject(pTmp, "addr_host", host); + cJSON_AddNumberToObject(pTmp, "addr_port", port); + } + cJSON_AddNumberToObject(pDestId, "vgId", pMsg->destId.vgId); + cJSON_AddItemToObject(pRoot, "destId", pDestId); + + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->term); + cJSON_AddStringToObject(pRoot, "term", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->startTime); + cJSON_AddStringToObject(pRoot, "startTime", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->beginIndex); + cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastIndex); + cJSON_AddStringToObject(pRoot, "lastIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, pMsg->lastConfigIndex); + cJSON_AddStringToObject(pRoot, "lastConfigIndex", u64buf); + cJSON_AddItemToObject(pRoot, "lastConfig", syncCfg2Json((SSyncCfg*)&(pMsg->lastConfig))); + + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pMsg->lastTerm); + cJSON_AddStringToObject(pRoot, "lastTerm", u64buf); + + cJSON_AddNumberToObject(pRoot, "seq", pMsg->seq); + + cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); + char* s; + s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data", s); + taosMemoryFree(s); + s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data2", s); + taosMemoryFree(s); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SyncSnapshotSend", pRoot); + return pJson; +} + +char* syncSnapshotSend2Str(const SyncSnapshotSend* pMsg) { + cJSON* pJson = syncSnapshotSend2Json(pMsg); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ---------------------- +void syncSnapshotSendPrint(const SyncSnapshotSend* pMsg) { + char* serialized = syncSnapshotSend2Str(pMsg); + printf("syncSnapshotSendPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncSnapshotSendPrint2(char* s, const SyncSnapshotSend* pMsg) { + char* serialized = syncSnapshotSend2Str(pMsg); + printf("syncSnapshotSendPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncSnapshotSendLog(const SyncSnapshotSend* pMsg) { + char* serialized = syncSnapshotSend2Str(pMsg); + sTrace("syncSnapshotSendLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncSnapshotSendLog2(char* s, const SyncSnapshotSend* pMsg) { + if (gRaftDetailLog) { + char* serialized = syncSnapshotSend2Str(pMsg); + sTrace("syncSnapshotSendLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +cJSON* syncClientRequestBatch2Json(const SyncClientRequestBatch* pMsg) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pMsg != NULL) { + cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); + cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); + cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); + cJSON_AddNumberToObject(pRoot, "dataLen", pMsg->dataLen); + cJSON_AddNumberToObject(pRoot, "dataCount", pMsg->dataCount); + + SRaftMeta* metaArr = syncClientRequestBatchMetaArr(pMsg); + SRpcMsg* msgArr = syncClientRequestBatchRpcMsgArr(pMsg); + + cJSON* pMetaArr = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "metaArr", pMetaArr); + for (int i = 0; i < pMsg->dataCount; ++i) { + cJSON* pMeta = cJSON_CreateObject(); + cJSON_AddNumberToObject(pMeta, "seqNum", metaArr[i].seqNum); + cJSON_AddNumberToObject(pMeta, "isWeak", metaArr[i].isWeak); + cJSON_AddItemToArray(pMetaArr, pMeta); + } + + cJSON* pMsgArr = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "msgArr", pMsgArr); + for (int i = 0; i < pMsg->dataCount; ++i) { + cJSON* pRpcMsgJson = syncRpcMsg2Json(&msgArr[i]); + cJSON_AddItemToArray(pMsgArr, pRpcMsgJson); + } + + char* s; + s = syncUtilPrintBin((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data", s); + taosMemoryFree(s); + s = syncUtilPrintBin2((char*)(pMsg->data), pMsg->dataLen); + cJSON_AddStringToObject(pRoot, "data2", s); + taosMemoryFree(s); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SyncClientRequestBatch", pRoot); + return pJson; +} + +char* syncClientRequestBatch2Str(const SyncClientRequestBatch* pMsg) { + cJSON* pJson = syncClientRequestBatch2Json(pMsg); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ---------------------- +void syncClientRequestBatchPrint(const SyncClientRequestBatch* pMsg) { + char* serialized = syncClientRequestBatch2Str(pMsg); + printf("syncClientRequestBatchPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncClientRequestBatchPrint2(char* s, const SyncClientRequestBatch* pMsg) { + char* serialized = syncClientRequestBatch2Str(pMsg); + printf("syncClientRequestBatchPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncClientRequestBatchLog(const SyncClientRequestBatch* pMsg) { + char* serialized = syncClientRequestBatch2Str(pMsg); + sTrace("syncClientRequestBatchLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncClientRequestBatchLog2(char* s, const SyncClientRequestBatch* pMsg) { + if (gRaftDetailLog) { + char* serialized = syncClientRequestBatch2Str(pMsg); + sLTrace("syncClientRequestBatchLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +// for debug ---------------------- +void syncClientRequestPrint(const SyncClientRequest* pMsg) { + char* serialized = syncClientRequest2Str(pMsg); + printf("syncClientRequestPrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncClientRequestPrint2(char* s, const SyncClientRequest* pMsg) { + char* serialized = syncClientRequest2Str(pMsg); + printf("syncClientRequestPrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncClientRequestLog(const SyncClientRequest* pMsg) { + char* serialized = syncClientRequest2Str(pMsg); + sTrace("syncClientRequestLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncClientRequestLog2(char* s, const SyncClientRequest* pMsg) { + if (gRaftDetailLog) { + char* serialized = syncClientRequest2Str(pMsg); + sTrace("syncClientRequestLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} diff --git a/source/libs/sync/test/sync_test_lib/src/syncRaftCfgDebug.c b/source/libs/sync/test/sync_test_lib/src/syncRaftCfgDebug.c new file mode 100644 index 0000000000000000000000000000000000000000..88b697fbf14e463772c48ac569e81876175624c4 --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/src/syncRaftCfgDebug.c @@ -0,0 +1,19 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "syncTest.h" + + diff --git a/source/libs/sync/test/sync_test_lib/src/syncRaftEntryDebug.c b/source/libs/sync/test/sync_test_lib/src/syncRaftEntryDebug.c new file mode 100644 index 0000000000000000000000000000000000000000..8179b24d29644e47abf7fc6d1807d37d561c490c --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/src/syncRaftEntryDebug.c @@ -0,0 +1,222 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "syncTest.h" + +cJSON* syncEntry2Json(const SSyncRaftEntry* pEntry) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pEntry != NULL) { + cJSON_AddNumberToObject(pRoot, "bytes", pEntry->bytes); + cJSON_AddNumberToObject(pRoot, "msgType", pEntry->msgType); + cJSON_AddNumberToObject(pRoot, "originalRpcType", pEntry->originalRpcType); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->seqNum); + cJSON_AddStringToObject(pRoot, "seqNum", u64buf); + cJSON_AddNumberToObject(pRoot, "isWeak", pEntry->isWeak); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->term); + cJSON_AddStringToObject(pRoot, "term", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->index); + cJSON_AddStringToObject(pRoot, "index", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, pEntry->rid); + cJSON_AddStringToObject(pRoot, "rid", u64buf); + cJSON_AddNumberToObject(pRoot, "dataLen", pEntry->dataLen); + + char* s; + s = syncUtilPrintBin((char*)(pEntry->data), pEntry->dataLen); + cJSON_AddStringToObject(pRoot, "data", s); + taosMemoryFree(s); + + s = syncUtilPrintBin2((char*)(pEntry->data), pEntry->dataLen); + cJSON_AddStringToObject(pRoot, "data2", s); + taosMemoryFree(s); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SSyncRaftEntry", pRoot); + return pJson; +} + +char* syncEntry2Str(const SSyncRaftEntry* pEntry) { + cJSON* pJson = syncEntry2Json(pEntry); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ---------------------- +void syncEntryPrint(const SSyncRaftEntry* pObj) { + char* serialized = syncEntry2Str(pObj); + printf("syncEntryPrint | len:%zu | %s \n", strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncEntryPrint2(char* s, const SSyncRaftEntry* pObj) { + char* serialized = syncEntry2Str(pObj); + printf("syncEntryPrint2 | len:%zu | %s | %s \n", strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncEntryLog(const SSyncRaftEntry* pObj) { + char* serialized = syncEntry2Str(pObj); + sTrace("syncEntryLog | len:%zu | %s", strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncEntryLog2(char* s, const SSyncRaftEntry* pObj) { + char* serialized = syncEntry2Str(pObj); + sTrace("syncEntryLog2 | len:%zu | %s | %s", strlen(serialized), s, serialized); + taosMemoryFree(serialized); +} + +//----------------------------------- +cJSON* raftCache2Json(SRaftEntryHashCache* pCache) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pCache != NULL) { + taosThreadMutexLock(&pCache->mutex); + + snprintf(u64buf, sizeof(u64buf), "%p", pCache->pSyncNode); + cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); + cJSON_AddNumberToObject(pRoot, "currentCount", pCache->currentCount); + cJSON_AddNumberToObject(pRoot, "maxCount", pCache->maxCount); + cJSON* pEntries = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "entries", pEntries); + + SSyncRaftEntry* pIter = (SSyncRaftEntry*)taosHashIterate(pCache->pEntryHash, NULL); + if (pIter != NULL) { + SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pIter; + cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); + } + while (pIter) { + pIter = taosHashIterate(pCache->pEntryHash, pIter); + if (pIter != NULL) { + SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pIter; + cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); + } + } + + taosThreadMutexUnlock(&pCache->mutex); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SRaftEntryHashCache", pRoot); + return pJson; +} + +char* raftCache2Str(SRaftEntryHashCache* pCache) { + cJSON* pJson = raftCache2Json(pCache); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +void raftCachePrint(SRaftEntryHashCache* pCache) { + char* serialized = raftCache2Str(pCache); + printf("raftCachePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftCachePrint2(char* s, SRaftEntryHashCache* pCache) { + char* serialized = raftCache2Str(pCache); + printf("raftCachePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftCacheLog(SRaftEntryHashCache* pCache) { + char* serialized = raftCache2Str(pCache); + sTrace("raftCacheLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void raftCacheLog2(char* s, SRaftEntryHashCache* pCache) { + if (gRaftDetailLog) { + char* serialized = raftCache2Str(pCache); + sLTrace("raftCacheLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +cJSON* raftEntryCache2Json(SRaftEntryCache* pCache) { + char u64buf[128] = {0}; + cJSON* pRoot = cJSON_CreateObject(); + + if (pCache != NULL) { + taosThreadMutexLock(&pCache->mutex); + + snprintf(u64buf, sizeof(u64buf), "%p", pCache->pSyncNode); + cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); + cJSON_AddNumberToObject(pRoot, "currentCount", pCache->currentCount); + cJSON_AddNumberToObject(pRoot, "maxCount", pCache->maxCount); + cJSON* pEntries = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "entries", pEntries); + + SSkipListIterator* pIter = tSkipListCreateIter(pCache->pSkipList); + while (tSkipListIterNext(pIter)) { + SSkipListNode* pNode = tSkipListIterGet(pIter); + ASSERT(pNode != NULL); + SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode); + cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); + } + tSkipListDestroyIter(pIter); + + taosThreadMutexUnlock(&pCache->mutex); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SRaftEntryCache", pRoot); + return pJson; +} + +char* raftEntryCache2Str(SRaftEntryCache* pObj) { + cJSON* pJson = raftEntryCache2Json(pObj); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +void raftEntryCachePrint(SRaftEntryCache* pObj) { + char* serialized = raftEntryCache2Str(pObj); + printf("raftEntryCachePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj) { + char* serialized = raftEntryCache2Str(pObj); + printf("raftEntryCachePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftEntryCacheLog(SRaftEntryCache* pObj) { + char* serialized = raftEntryCache2Str(pObj); + sTrace("raftEntryCacheLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj) { + if (gRaftDetailLog) { + char* serialized = raftEntryCache2Str(pObj); + sLTrace("raftEntryCacheLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} diff --git a/source/libs/sync/test/sync_test_lib/src/syncRaftLogDebug.c b/source/libs/sync/test/sync_test_lib/src/syncRaftLogDebug.c new file mode 100644 index 0000000000000000000000000000000000000000..53e414ccfd8824dce7d0cff7779c67d40ff93cd9 --- /dev/null +++ b/source/libs/sync/test/sync_test_lib/src/syncRaftLogDebug.c @@ -0,0 +1,182 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "syncTest.h" + +cJSON* logStore2Json(SSyncLogStore* pLogStore) { + char u64buf[128] = {0}; + SSyncLogStoreData* pData = (SSyncLogStoreData*)pLogStore->data; + cJSON* pRoot = cJSON_CreateObject(); + + if (pData != NULL && pData->pWal != NULL) { + snprintf(u64buf, sizeof(u64buf), "%p", pData->pSyncNode); + cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pData->pWal); + cJSON_AddStringToObject(pRoot, "pWal", u64buf); + + SyncIndex beginIndex = raftLogBeginIndex(pLogStore); + snprintf(u64buf, sizeof(u64buf), "%" PRId64, beginIndex); + cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); + + SyncIndex endIndex = raftLogEndIndex(pLogStore); + snprintf(u64buf, sizeof(u64buf), "%" PRId64, endIndex); + cJSON_AddStringToObject(pRoot, "endIndex", u64buf); + + int32_t count = raftLogEntryCount(pLogStore); + cJSON_AddNumberToObject(pRoot, "entryCount", count); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogWriteIndex(pLogStore)); + cJSON_AddStringToObject(pRoot, "WriteIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%d", raftLogIsEmpty(pLogStore)); + cJSON_AddStringToObject(pRoot, "IsEmpty", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogLastIndex(pLogStore)); + cJSON_AddStringToObject(pRoot, "LastIndex", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, raftLogLastTerm(pLogStore)); + cJSON_AddStringToObject(pRoot, "LastTerm", u64buf); + + cJSON* pEntries = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "pEntries", pEntries); + + if (!raftLogIsEmpty(pLogStore)) { + for (SyncIndex i = beginIndex; i <= endIndex; ++i) { + SSyncRaftEntry* pEntry = NULL; + raftLogGetEntry(pLogStore, i, &pEntry); + + cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry)); + syncEntryDestory(pEntry); + } + } + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SSyncLogStore", pRoot); + return pJson; +} + +char* logStore2Str(SSyncLogStore* pLogStore) { + cJSON* pJson = logStore2Json(pLogStore); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +cJSON* logStoreSimple2Json(SSyncLogStore* pLogStore) { + char u64buf[128] = {0}; + SSyncLogStoreData* pData = (SSyncLogStoreData*)pLogStore->data; + cJSON* pRoot = cJSON_CreateObject(); + + if (pData != NULL && pData->pWal != NULL) { + snprintf(u64buf, sizeof(u64buf), "%p", pData->pSyncNode); + cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf); + snprintf(u64buf, sizeof(u64buf), "%p", pData->pWal); + cJSON_AddStringToObject(pRoot, "pWal", u64buf); + + SyncIndex beginIndex = raftLogBeginIndex(pLogStore); + snprintf(u64buf, sizeof(u64buf), "%" PRId64, beginIndex); + cJSON_AddStringToObject(pRoot, "beginIndex", u64buf); + + SyncIndex endIndex = raftLogEndIndex(pLogStore); + snprintf(u64buf, sizeof(u64buf), "%" PRId64, endIndex); + cJSON_AddStringToObject(pRoot, "endIndex", u64buf); + + int32_t count = raftLogEntryCount(pLogStore); + cJSON_AddNumberToObject(pRoot, "entryCount", count); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogWriteIndex(pLogStore)); + cJSON_AddStringToObject(pRoot, "WriteIndex", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%d", raftLogIsEmpty(pLogStore)); + cJSON_AddStringToObject(pRoot, "IsEmpty", u64buf); + + snprintf(u64buf, sizeof(u64buf), "%" PRId64, raftLogLastIndex(pLogStore)); + cJSON_AddStringToObject(pRoot, "LastIndex", u64buf); + snprintf(u64buf, sizeof(u64buf), "%" PRIu64, raftLogLastTerm(pLogStore)); + cJSON_AddStringToObject(pRoot, "LastTerm", u64buf); + } + + cJSON* pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SSyncLogStoreSimple", pRoot); + return pJson; +} + +char* logStoreSimple2Str(SSyncLogStore* pLogStore) { + cJSON* pJson = logStoreSimple2Json(pLogStore); + char* serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +// for debug ----------------- +void logStorePrint(SSyncLogStore* pLogStore) { + char* serialized = logStore2Str(pLogStore); + printf("logStorePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void logStorePrint2(char* s, SSyncLogStore* pLogStore) { + char* serialized = logStore2Str(pLogStore); + printf("logStorePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void logStoreLog(SSyncLogStore* pLogStore) { + if (gRaftDetailLog) { + char* serialized = logStore2Str(pLogStore); + sLTrace("logStoreLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); + } +} + +void logStoreLog2(char* s, SSyncLogStore* pLogStore) { + if (gRaftDetailLog) { + char* serialized = logStore2Str(pLogStore); + sLTrace("logStoreLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +} + +// for debug ----------------- +void logStoreSimplePrint(SSyncLogStore* pLogStore) { + char* serialized = logStoreSimple2Str(pLogStore); + printf("logStoreSimplePrint | len:%d | %s \n", (int32_t)strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void logStoreSimplePrint2(char* s, SSyncLogStore* pLogStore) { + char* serialized = logStoreSimple2Str(pLogStore); + printf("logStoreSimplePrint2 | len:%d | %s | %s \n", (int32_t)strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void logStoreSimpleLog(SSyncLogStore* pLogStore) { + char* serialized = logStoreSimple2Str(pLogStore); + sTrace("logStoreSimpleLog | len:%d | %s", (int32_t)strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void logStoreSimpleLog2(char* s, SSyncLogStore* pLogStore) { + if (gRaftDetailLog) { + char* serialized = logStoreSimple2Str(pLogStore); + sTrace("logStoreSimpleLog2 | len:%d | %s | %s", (int32_t)strlen(serialized), s, serialized); + taosMemoryFree(serialized); + } +}