diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index e553dff270..24d259b0f2 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -206,6 +206,19 @@ enum { TD_DEF_MSG_TYPE(TDMT_VND_CANCEL_SMA, "vnode-cancel-sma", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_DROP_SMA, "vnode-drop-sma", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_TIMEOUT, "vnode-sync-timeout", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_PING, "vnode-sync-ping", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_PING_REPLY, "vnode-sync-ping-reply", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_CLIENT_REQUEST, "vnode-sync-client-request", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_CLIENT_REQUEST_REPLY, "vnode-sync-client-request-reply", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_REQUEST_VOTE, "vnode-sync-request-vote", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_REQUEST_VOTE_REPLY, "vnode-sync-request-vote-reply", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_APPEND_ENTRIES, "vnode-sync-append-entries", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_APPEND_ENTRIES_REPLY, "vnode-sync-append-entries-reply", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_NOOP, "vnode-sync-noop", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_UNKNOWN, "vnode-sync-unknown", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_SYNC_COMMON_RESPONSE, "vnode-sync-common-response", NULL, NULL) + // Requests handled by QNODE TD_NEW_MSG_SEG(TDMT_QND_MSG) diff --git a/include/libs/sync/sync.h b/include/libs/sync/sync.h index a2f88490f0..86666f4c68 100644 --- a/include/libs/sync/sync.h +++ b/include/libs/sync/sync.h @@ -22,6 +22,7 @@ extern "C" { #include #include +#include "cJSON.h" #include "taosdef.h" #include "trpc.h" #include "wal.h" @@ -38,14 +39,9 @@ typedef enum { TAOS_SYNC_STATE_ERROR = 103, } ESyncState; -typedef struct SSyncBuffer { - void* data; - size_t len; -} SSyncBuffer; - typedef struct SNodeInfo { - uint16_t nodePort; // node sync Port - char nodeFqdn[TSDB_FQDN_LEN]; // node FQDN + uint16_t nodePort; + char nodeFqdn[TSDB_FQDN_LEN]; } SNodeInfo; typedef struct SSyncCfg { @@ -54,40 +50,37 @@ typedef struct SSyncCfg { SNodeInfo nodeInfo[TSDB_MAX_REPLICA]; } SSyncCfg; -typedef struct SNodesRole { - int32_t replicaNum; - SNodeInfo nodeInfo[TSDB_MAX_REPLICA]; - ESyncState role[TSDB_MAX_REPLICA]; -} SNodesRole; +typedef struct SRaftId { + SyncNodeId addr; + SyncGroupId vgId; +} SRaftId; -// abstract definition of snapshot typedef struct SSnapshot { void* data; SyncIndex lastApplyIndex; } SSnapshot; +typedef enum { + TAOS_SYNC_FSM_CB_SUCCESS = 0, + TAOS_SYNC_FSM_CB_OTHER_ERROR, +} ESyncFsmCbCode; + +typedef struct SFsmCbMeta { + SyncIndex index; + bool isWeak; + int32_t code; + ESyncState state; + uint64_t seqNum; +} SFsmCbMeta; + typedef struct SSyncFSM { void* data; - // when value in pMsg finish a raft flow, FpCommitCb is called, code indicates the result - // user can do something according to the code and isWeak. for example, write data into tsdb - void (*FpCommitCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state); + void (*FpCommitCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SFsmCbMeta cbMeta); + void (*FpPreCommitCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SFsmCbMeta cbMeta); + void (*FpRollBackCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SFsmCbMeta cbMeta); - // when value in pMsg has been written into local log store, FpPreCommitCb is called, code indicates the result - // user can do something according to the code and isWeak. for example, write data into tsdb - void (*FpPreCommitCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state); - - // when log entry is updated by a new one, FpRollBackCb is called - // user can do something to roll back. for example, delete data from tsdb, or just ignore it - void (*FpRollBackCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state); - - // user should implement this function, use "data" to take snapshot into "snapshot" int32_t (*FpTakeSnapshot)(SSnapshot* snapshot); - - // user should implement this function, restore "data" from "snapshot" int32_t (*FpRestoreSnapshot)(const SSnapshot* snapshot); } SSyncFSM; @@ -123,21 +116,6 @@ typedef struct SSyncLogStore { } SSyncLogStore; -// raft need to persist two variables in storage: currentTerm, voteFor -typedef struct SStateMgr { - void* data; - - int32_t (*getCurrentTerm)(struct SStateMgr* pMgr, SyncTerm* pCurrentTerm); - int32_t (*persistCurrentTerm)(struct SStateMgr* pMgr, SyncTerm pCurrentTerm); - - int32_t (*getVoteFor)(struct SStateMgr* pMgr, SyncNodeId* pVoteFor); - int32_t (*persistVoteFor)(struct SStateMgr* pMgr, SyncNodeId voteFor); - - int32_t (*getSyncCfg)(struct SStateMgr* pMgr, SSyncCfg* pSyncCfg); - int32_t (*persistSyncCfg)(struct SStateMgr* pMgr, SSyncCfg* pSyncCfg); - -} SStateMgr; - typedef struct SSyncInfo { SyncGroupId vgId; SSyncCfg syncCfg; @@ -152,25 +130,33 @@ typedef struct SSyncInfo { } SSyncInfo; +int32_t syncInit(); +void syncCleanUp(); +int64_t syncOpen(const SSyncInfo* pSyncInfo); +void syncStart(int64_t rid); +void syncStop(int64_t rid); +int32_t syncReconfig(int64_t rid, const SSyncCfg* pSyncCfg); +ESyncState syncGetMyRole(int64_t rid); +const char* syncGetMyRoleStr(int64_t rid); +SyncTerm syncGetMyTerm(int64_t rid); + +typedef enum { + TAOS_SYNC_PROPOSE_SUCCESS = 0, + TAOS_SYNC_PROPOSE_NOT_LEADER, + TAOS_SYNC_PROPOSE_OTHER_ERROR, +} ESyncProposeCode; + +int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak); + +extern int32_t sDebugFlag; + +//----------------------------------------- struct SSyncNode; typedef struct SSyncNode SSyncNode; -int32_t syncInit(); -void syncCleanUp(); - -int64_t syncStart(const SSyncInfo* pSyncInfo); -void syncStop(int64_t rid); -int32_t syncReconfig(int64_t rid, const SSyncCfg* pSyncCfg); -int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak); -ESyncState syncGetMyRole(int64_t rid); - -// propose with sequence number, to implement linearizable semantics -int32_t syncPropose2(int64_t rid, const SRpcMsg* pMsg, bool isWeak, uint64_t seqNum); - -// for compatibility, the same as syncPropose -int32_t syncForwardToPeer(int64_t rid, const SRpcMsg* pMsg, bool isWeak); - -extern int32_t sDebugFlag; +struct SSyncBuffer; +typedef struct SSyncBuffer SSyncBuffer; +//----------------------------------------- #ifdef __cplusplus } diff --git a/include/libs/sync/syncTools.h b/include/libs/sync/syncTools.h new file mode 100644 index 0000000000..6d28f54e29 --- /dev/null +++ b/include/libs/sync/syncTools.h @@ -0,0 +1,376 @@ +/* + * 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_TOOLS_H +#define _TD_LIBS_SYNC_TOOLS_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include +#include +#include "cJSON.h" +#include "taosdef.h" +#include "trpc.h" +#include "wal.h" + +// ------------------ control ------------------- +struct SSyncNode; +typedef struct SSyncNode SSyncNode; + +SSyncNode* syncNodeAcquire(int64_t rid); +void syncNodeRelease(SSyncNode* pNode); + +int32_t syncGetRespRpc(int64_t rid, uint64_t index, SRpcMsg* msg); +int32_t syncGetAndDelRespRpc(int64_t rid, uint64_t index, SRpcMsg* msg); +void syncSetQ(int64_t rid, void* queueHandle); +void syncSetRpc(int64_t rid, void* rpcHandle); +char* sync2SimpleStr(int64_t rid); + +// set timer ms +void setPingTimerMS(int64_t rid, int32_t pingTimerMS); +void setElectTimerMS(int64_t rid, int32_t electTimerMS); +void setHeartbeatTimerMS(int64_t rid, int32_t hbTimerMS); + +// for compatibility, the same as syncPropose +int32_t syncForwardToPeer(int64_t rid, const SRpcMsg* pMsg, bool isWeak); + +// utils +const char* syncUtilState2String(ESyncState state); + +// ------------------ for debug ------------------- +void syncRpcMsgPrint(SRpcMsg* pMsg); +void syncRpcMsgPrint2(char* s, SRpcMsg* pMsg); +void syncRpcMsgLog(SRpcMsg* pMsg); +void syncRpcMsgLog2(char* s, SRpcMsg* pMsg); + +// ------------------ for compile ------------------- +typedef struct SSyncBuffer { + void* data; + size_t len; +} SSyncBuffer; + +typedef struct SNodesRole { + int32_t replicaNum; + SNodeInfo nodeInfo[TSDB_MAX_REPLICA]; + ESyncState role[TSDB_MAX_REPLICA]; +} SNodesRole; + +typedef struct SStateMgr { + void* data; + + int32_t (*getCurrentTerm)(struct SStateMgr* pMgr, SyncTerm* pCurrentTerm); + int32_t (*persistCurrentTerm)(struct SStateMgr* pMgr, SyncTerm pCurrentTerm); + + int32_t (*getVoteFor)(struct SStateMgr* pMgr, SyncNodeId* pVoteFor); + int32_t (*persistVoteFor)(struct SStateMgr* pMgr, SyncNodeId voteFor); + + int32_t (*getSyncCfg)(struct SStateMgr* pMgr, SSyncCfg* pSyncCfg); + int32_t (*persistSyncCfg)(struct SStateMgr* pMgr, SSyncCfg* pSyncCfg); + +} SStateMgr; + +// ------------------ for message process ------------------- + +// --------------------------------------------- +typedef struct SyncPing { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + uint32_t dataLen; + char data[]; +} SyncPing; + +SyncPing* syncPingBuild(uint32_t dataLen); +SyncPing* syncPingBuild2(const SRaftId* srcId, const SRaftId* destId, int32_t vgId, const char* str); +SyncPing* syncPingBuild3(const SRaftId* srcId, const SRaftId* destId, int32_t vgId); +void syncPingDestroy(SyncPing* pMsg); +void syncPingSerialize(const SyncPing* pMsg, char* buf, uint32_t bufLen); +void syncPingDeserialize(const char* buf, uint32_t len, SyncPing* pMsg); +char* syncPingSerialize2(const SyncPing* pMsg, uint32_t* len); +SyncPing* syncPingDeserialize2(const char* buf, uint32_t len); +int32_t syncPingSerialize3(const SyncPing* pMsg, char* buf, int32_t bufLen); +SyncPing* syncPingDeserialize3(void* buf, int32_t bufLen); +void syncPing2RpcMsg(const SyncPing* pMsg, SRpcMsg* pRpcMsg); +void syncPingFromRpcMsg(const SRpcMsg* pRpcMsg, SyncPing* pMsg); +SyncPing* syncPingFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncPing2Json(const SyncPing* pMsg); +char* syncPing2Str(const SyncPing* pMsg); + +// for debug ---------------------- +void syncPingPrint(const SyncPing* pMsg); +void syncPingPrint2(char* s, const SyncPing* pMsg); +void syncPingLog(const SyncPing* pMsg); +void syncPingLog2(char* s, const SyncPing* pMsg); + +// --------------------------------------------- +typedef struct SyncPingReply { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + uint32_t dataLen; + char data[]; +} SyncPingReply; + +SyncPingReply* syncPingReplyBuild(uint32_t dataLen); +SyncPingReply* syncPingReplyBuild2(const SRaftId* srcId, const SRaftId* destId, int32_t vgId, const char* str); +SyncPingReply* syncPingReplyBuild3(const SRaftId* srcId, const SRaftId* destId, int32_t vgId); +void syncPingReplyDestroy(SyncPingReply* pMsg); +void syncPingReplySerialize(const SyncPingReply* pMsg, char* buf, uint32_t bufLen); +void syncPingReplyDeserialize(const char* buf, uint32_t len, SyncPingReply* pMsg); +char* syncPingReplySerialize2(const SyncPingReply* pMsg, uint32_t* len); +SyncPingReply* syncPingReplyDeserialize2(const char* buf, uint32_t len); +int32_t syncPingReplySerialize3(const SyncPingReply* pMsg, char* buf, int32_t bufLen); +SyncPingReply* syncPingReplyDeserialize3(void* buf, int32_t bufLen); +void syncPingReply2RpcMsg(const SyncPingReply* pMsg, SRpcMsg* pRpcMsg); +void syncPingReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncPingReply* pMsg); +SyncPingReply* syncPingReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncPingReply2Json(const SyncPingReply* pMsg); +char* syncPingReply2Str(const SyncPingReply* pMsg); + +// for debug ---------------------- +void syncPingReplyPrint(const SyncPingReply* pMsg); +void syncPingReplyPrint2(char* s, const SyncPingReply* pMsg); +void syncPingReplyLog(const SyncPingReply* pMsg); +void syncPingReplyLog2(char* s, const SyncPingReply* pMsg); + +// --------------------------------------------- +typedef enum ESyncTimeoutType { + SYNC_TIMEOUT_PING = 100, + SYNC_TIMEOUT_ELECTION, + SYNC_TIMEOUT_HEARTBEAT, +} ESyncTimeoutType; + +typedef struct SyncTimeout { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + ESyncTimeoutType timeoutType; + uint64_t logicClock; + int32_t timerMS; + void* data; // need optimized +} SyncTimeout; + +SyncTimeout* syncTimeoutBuild(); +SyncTimeout* syncTimeoutBuild2(ESyncTimeoutType timeoutType, uint64_t logicClock, int32_t timerMS, int32_t vgId, + void* data); +void syncTimeoutDestroy(SyncTimeout* pMsg); +void syncTimeoutSerialize(const SyncTimeout* pMsg, char* buf, uint32_t bufLen); +void syncTimeoutDeserialize(const char* buf, uint32_t len, SyncTimeout* pMsg); +char* syncTimeoutSerialize2(const SyncTimeout* pMsg, uint32_t* len); +SyncTimeout* syncTimeoutDeserialize2(const char* buf, uint32_t len); +void syncTimeout2RpcMsg(const SyncTimeout* pMsg, SRpcMsg* pRpcMsg); +void syncTimeoutFromRpcMsg(const SRpcMsg* pRpcMsg, SyncTimeout* pMsg); +SyncTimeout* syncTimeoutFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncTimeout2Json(const SyncTimeout* pMsg); +char* syncTimeout2Str(const SyncTimeout* pMsg); + +// for debug ---------------------- +void syncTimeoutPrint(const SyncTimeout* pMsg); +void syncTimeoutPrint2(char* s, const SyncTimeout* pMsg); +void syncTimeoutLog(const SyncTimeout* pMsg); +void syncTimeoutLog2(char* s, const SyncTimeout* pMsg); + +// --------------------------------------------- +typedef struct SyncClientRequest { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; // SyncClientRequest msgType + uint32_t originalRpcType; // user RpcMsg msgType + uint64_t seqNum; + bool isWeak; + uint32_t dataLen; // user RpcMsg.contLen + char data[]; // user RpcMsg.pCont +} SyncClientRequest; + +SyncClientRequest* syncClientRequestBuild(uint32_t dataLen); +SyncClientRequest* syncClientRequestBuild2(const SRpcMsg* pOriginalRpcMsg, 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); + +// for debug ---------------------- +void syncClientRequestPrint(const SyncClientRequest* pMsg); +void syncClientRequestPrint2(char* s, const SyncClientRequest* pMsg); +void syncClientRequestLog(const SyncClientRequest* pMsg); +void syncClientRequestLog2(char* s, const SyncClientRequest* pMsg); + +// --------------------------------------------- +typedef struct SyncClientRequestReply { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + int32_t errCode; + SRaftId leaderHint; +} SyncClientRequestReply; + +// --------------------------------------------- +typedef struct SyncRequestVote { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + SyncTerm term; + SyncIndex lastLogIndex; + SyncTerm lastLogTerm; +} SyncRequestVote; + +SyncRequestVote* syncRequestVoteBuild(int32_t vgId); +void syncRequestVoteDestroy(SyncRequestVote* pMsg); +void syncRequestVoteSerialize(const SyncRequestVote* pMsg, char* buf, uint32_t bufLen); +void syncRequestVoteDeserialize(const char* buf, uint32_t len, SyncRequestVote* pMsg); +char* syncRequestVoteSerialize2(const SyncRequestVote* pMsg, uint32_t* len); +SyncRequestVote* syncRequestVoteDeserialize2(const char* buf, uint32_t len); +void syncRequestVote2RpcMsg(const SyncRequestVote* pMsg, SRpcMsg* pRpcMsg); +void syncRequestVoteFromRpcMsg(const SRpcMsg* pRpcMsg, SyncRequestVote* pMsg); +SyncRequestVote* syncRequestVoteFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncRequestVote2Json(const SyncRequestVote* pMsg); +char* syncRequestVote2Str(const SyncRequestVote* pMsg); + +// for debug ---------------------- +void syncRequestVotePrint(const SyncRequestVote* pMsg); +void syncRequestVotePrint2(char* s, const SyncRequestVote* pMsg); +void syncRequestVoteLog(const SyncRequestVote* pMsg); +void syncRequestVoteLog2(char* s, const SyncRequestVote* pMsg); + +// --------------------------------------------- +typedef struct SyncRequestVoteReply { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + SyncTerm term; + bool voteGranted; +} SyncRequestVoteReply; + +SyncRequestVoteReply* syncRequestVoteReplyBuild(int32_t vgId); +void syncRequestVoteReplyDestroy(SyncRequestVoteReply* pMsg); +void syncRequestVoteReplySerialize(const SyncRequestVoteReply* pMsg, char* buf, uint32_t bufLen); +void syncRequestVoteReplyDeserialize(const char* buf, uint32_t len, SyncRequestVoteReply* pMsg); +char* syncRequestVoteReplySerialize2(const SyncRequestVoteReply* pMsg, uint32_t* len); +SyncRequestVoteReply* syncRequestVoteReplyDeserialize2(const char* buf, uint32_t len); +void syncRequestVoteReply2RpcMsg(const SyncRequestVoteReply* pMsg, SRpcMsg* pRpcMsg); +void syncRequestVoteReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncRequestVoteReply* pMsg); +SyncRequestVoteReply* syncRequestVoteReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncRequestVoteReply2Json(const SyncRequestVoteReply* pMsg); +char* syncRequestVoteReply2Str(const SyncRequestVoteReply* pMsg); + +// for debug ---------------------- +void syncRequestVoteReplyPrint(const SyncRequestVoteReply* pMsg); +void syncRequestVoteReplyPrint2(char* s, const SyncRequestVoteReply* pMsg); +void syncRequestVoteReplyLog(const SyncRequestVoteReply* pMsg); +void syncRequestVoteReplyLog2(char* s, const SyncRequestVoteReply* pMsg); + +// --------------------------------------------- +typedef struct SyncAppendEntries { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + SyncTerm term; + SyncIndex prevLogIndex; + SyncTerm prevLogTerm; + SyncIndex commitIndex; + uint32_t dataLen; + char data[]; +} SyncAppendEntries; + +SyncAppendEntries* syncAppendEntriesBuild(uint32_t dataLen, int32_t vgId); +void syncAppendEntriesDestroy(SyncAppendEntries* pMsg); +void syncAppendEntriesSerialize(const SyncAppendEntries* pMsg, char* buf, uint32_t bufLen); +void syncAppendEntriesDeserialize(const char* buf, uint32_t len, SyncAppendEntries* pMsg); +char* syncAppendEntriesSerialize2(const SyncAppendEntries* pMsg, uint32_t* len); +SyncAppendEntries* syncAppendEntriesDeserialize2(const char* buf, uint32_t len); +void syncAppendEntries2RpcMsg(const SyncAppendEntries* pMsg, SRpcMsg* pRpcMsg); +void syncAppendEntriesFromRpcMsg(const SRpcMsg* pRpcMsg, SyncAppendEntries* pMsg); +SyncAppendEntries* syncAppendEntriesFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncAppendEntries2Json(const SyncAppendEntries* pMsg); +char* syncAppendEntries2Str(const SyncAppendEntries* pMsg); + +// for debug ---------------------- +void syncAppendEntriesPrint(const SyncAppendEntries* pMsg); +void syncAppendEntriesPrint2(char* s, const SyncAppendEntries* pMsg); +void syncAppendEntriesLog(const SyncAppendEntries* pMsg); +void syncAppendEntriesLog2(char* s, const SyncAppendEntries* pMsg); + +// --------------------------------------------- +typedef struct SyncAppendEntriesReply { + uint32_t bytes; + int32_t vgId; + uint32_t msgType; + SRaftId srcId; + SRaftId destId; + // private data + SyncTerm term; + bool success; + SyncIndex matchIndex; +} SyncAppendEntriesReply; + +SyncAppendEntriesReply* syncAppendEntriesReplyBuild(int32_t vgId); +void syncAppendEntriesReplyDestroy(SyncAppendEntriesReply* pMsg); +void syncAppendEntriesReplySerialize(const SyncAppendEntriesReply* pMsg, char* buf, uint32_t bufLen); +void syncAppendEntriesReplyDeserialize(const char* buf, uint32_t len, SyncAppendEntriesReply* pMsg); +char* syncAppendEntriesReplySerialize2(const SyncAppendEntriesReply* pMsg, uint32_t* len); +SyncAppendEntriesReply* syncAppendEntriesReplyDeserialize2(const char* buf, uint32_t len); +void syncAppendEntriesReply2RpcMsg(const SyncAppendEntriesReply* pMsg, SRpcMsg* pRpcMsg); +void syncAppendEntriesReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncAppendEntriesReply* pMsg); +SyncAppendEntriesReply* syncAppendEntriesReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); +cJSON* syncAppendEntriesReply2Json(const SyncAppendEntriesReply* pMsg); +char* syncAppendEntriesReply2Str(const SyncAppendEntriesReply* pMsg); + +// for debug ---------------------- +void syncAppendEntriesReplyPrint(const SyncAppendEntriesReply* pMsg); +void syncAppendEntriesReplyPrint2(char* s, const SyncAppendEntriesReply* pMsg); +void syncAppendEntriesReplyLog(const SyncAppendEntriesReply* pMsg); +void syncAppendEntriesReplyLog2(char* s, const SyncAppendEntriesReply* pMsg); + +// on message ---------------------- +int32_t syncNodeOnPingCb(SSyncNode* ths, SyncPing* pMsg); +int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg); +int32_t syncNodeOnTimeoutCb(SSyncNode* ths, SyncTimeout* pMsg); +int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg); +int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg); +int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg); +int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg); +int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* pMsg); + +//--------------------- + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_LIBS_SYNC_TOOLS_H*/ diff --git a/source/libs/sync/CMakeLists.txt b/source/libs/sync/CMakeLists.txt index 551849c6f2..cb196acc02 100644 --- a/source/libs/sync/CMakeLists.txt +++ b/source/libs/sync/CMakeLists.txt @@ -11,7 +11,7 @@ target_link_libraries( target_include_directories( sync - PUBLIC "${TD_SOURCE_DIR}/include/libs/sync" + PUBLIC "${CMAKE_SOURCE_DIR}/include/libs/sync" PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc" ) diff --git a/source/libs/sync/inc/syncEnv.h b/source/libs/sync/inc/syncEnv.h index 7940d7f436..e9550ff989 100644 --- a/source/libs/sync/inc/syncEnv.h +++ b/source/libs/sync/inc/syncEnv.h @@ -31,10 +31,10 @@ extern "C" { #define TIMER_MAX_MS 0x7FFFFFFF #define ENV_TICK_TIMER_MS 1000 #define PING_TIMER_MS 1000 -#define ELECT_TIMER_MS_MIN 150 +#define ELECT_TIMER_MS_MIN 500 #define ELECT_TIMER_MS_MAX (ELECT_TIMER_MS_MIN * 2) #define ELECT_TIMER_MS_RANGE (ELECT_TIMER_MS_MAX - ELECT_TIMER_MS_MIN) -#define HEARTBEAT_TIMER_MS 30 +#define HEARTBEAT_TIMER_MS 100 #define EMPTY_RAFT_ID ((SRaftId){.addr = 0, .vgId = 0}) diff --git a/source/libs/sync/inc/syncInt.h b/source/libs/sync/inc/syncInt.h index cfa87048a7..cb539e8379 100644 --- a/source/libs/sync/inc/syncInt.h +++ b/source/libs/sync/inc/syncInt.h @@ -25,6 +25,7 @@ extern "C" { #include #include "cJSON.h" #include "sync.h" +#include "syncTools.h" #include "taosdef.h" #include "tglobal.h" #include "tlog.h" @@ -67,6 +68,43 @@ extern "C" { } \ } +#define sFatalLong(...) \ + { \ + if (sDebugFlag & DEBUG_FATAL) { \ + taosPrintLongString("SYN FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); \ + } \ + } +#define sErrorLong(...) \ + { \ + if (sDebugFlag & DEBUG_ERROR) { \ + taosPrintLongString("SYN ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); \ + } \ + } +#define sWarnLong(...) \ + { \ + if (sDebugFlag & DEBUG_WARN) { \ + taosPrintLongString("SYN WARN ", DEBUG_WARN, 255, __VA_ARGS__); \ + } \ + } +#define sInfoLong(...) \ + { \ + if (sDebugFlag & DEBUG_INFO) { \ + taosPrintLongString("SYN INFO ", DEBUG_INFO, 255, __VA_ARGS__); \ + } \ + } +#define sDebugLong(...) \ + { \ + if (sDebugFlag & DEBUG_DEBUG) { \ + taosPrintLongString("SYN DEBUG ", DEBUG_DEBUG, sDebugFlag, __VA_ARGS__); \ + } \ + } +#define sTraceLong(...) \ + { \ + if (sDebugFlag & DEBUG_TRACE) { \ + taosPrintLongString("SYN TRACE ", DEBUG_TRACE, sDebugFlag, __VA_ARGS__); \ + } \ + } + struct SyncTimeout; typedef struct SyncTimeout SyncTimeout; @@ -106,17 +144,19 @@ typedef struct SVotesRespond SVotesRespond; struct SSyncIndexMgr; typedef struct SSyncIndexMgr SSyncIndexMgr; -typedef struct SRaftId { - SyncNodeId addr; // typedef uint64_t SyncNodeId; - SyncGroupId vgId; // typedef int32_t SyncGroupId; -} SRaftId; +struct SRaftCfg; +typedef struct SRaftCfg SRaftCfg; + +struct SSyncRespMgr; +typedef struct SSyncRespMgr SSyncRespMgr; typedef struct SSyncNode { // init by SSyncInfo SyncGroupId vgId; - SSyncCfg syncCfg; + SRaftCfg* pRaftCfg; char path[TSDB_FILENAME_LEN]; char raftStorePath[TSDB_FILENAME_LEN * 2]; + char configPath[TSDB_FILENAME_LEN * 2]; // sync io SWal* pWal; @@ -199,10 +239,14 @@ typedef struct SSyncNode { int32_t (*FpOnAppendEntriesReply)(SSyncNode* ths, SyncAppendEntriesReply* pMsg); int32_t (*FpOnTimeout)(SSyncNode* pSyncNode, SyncTimeout* pMsg); + // tools + SSyncRespMgr* pSyncRespMgr; + } SSyncNode; // open/close -------------- SSyncNode* syncNodeOpen(const SSyncInfo* pSyncInfo); +void syncNodeStart(SSyncNode* pSyncNode); void syncNodeClose(SSyncNode* pSyncNode); // ping -------------- @@ -222,10 +266,12 @@ int32_t syncNodeStartHeartbeatTimer(SSyncNode* pSyncNode); int32_t syncNodeStopHeartbeatTimer(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); +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); + SSyncNode* syncNodeAcquire(int64_t rid); void syncNodeRelease(SSyncNode* pNode); diff --git a/source/libs/sync/inc/syncMessage.h b/source/libs/sync/inc/syncMessage.h index ffb2ae3a26..19079d518d 100644 --- a/source/libs/sync/inc/syncMessage.h +++ b/source/libs/sync/inc/syncMessage.h @@ -27,22 +27,6 @@ extern "C" { #include "syncInt.h" #include "taosdef.h" -// encode as uint32 -typedef enum ESyncMessageType { - SYNC_UNKNOWN = 9999, - SYNC_TIMEOUT = 99, - SYNC_PING = 101, - SYNC_PING_REPLY = 103, - SYNC_CLIENT_REQUEST = 105, - SYNC_CLIENT_REQUEST_REPLY = 107, - SYNC_REQUEST_VOTE = 109, - SYNC_REQUEST_VOTE_REPLY = 111, - SYNC_APPEND_ENTRIES = 113, - SYNC_APPEND_ENTRIES_REPLY = 115, - SYNC_RESPONSE = 119, - -} ESyncMessageType; - // --------------------------------------------- cJSON* syncRpcMsg2Json(SRpcMsg* pRpcMsg); cJSON* syncRpcUnknownMsg2Json(); @@ -53,263 +37,7 @@ void syncRpcMsgPrint(SRpcMsg* pMsg); void syncRpcMsgPrint2(char* s, SRpcMsg* pMsg); void syncRpcMsgLog(SRpcMsg* pMsg); void syncRpcMsgLog2(char* s, SRpcMsg* pMsg); - // --------------------------------------------- -typedef enum ESyncTimeoutType { - SYNC_TIMEOUT_PING = 100, - SYNC_TIMEOUT_ELECTION, - SYNC_TIMEOUT_HEARTBEAT, -} ESyncTimeoutType; - -typedef struct SyncTimeout { - uint32_t bytes; - uint32_t msgType; - ESyncTimeoutType timeoutType; - uint64_t logicClock; - int32_t timerMS; - void* data; // need optimized -} SyncTimeout; - -SyncTimeout* syncTimeoutBuild(); -SyncTimeout* syncTimeoutBuild2(ESyncTimeoutType timeoutType, uint64_t logicClock, int32_t timerMS, void* data); -void syncTimeoutDestroy(SyncTimeout* pMsg); -void syncTimeoutSerialize(const SyncTimeout* pMsg, char* buf, uint32_t bufLen); -void syncTimeoutDeserialize(const char* buf, uint32_t len, SyncTimeout* pMsg); -char* syncTimeoutSerialize2(const SyncTimeout* pMsg, uint32_t* len); // -SyncTimeout* syncTimeoutDeserialize2(const char* buf, uint32_t len); // -void syncTimeout2RpcMsg(const SyncTimeout* pMsg, SRpcMsg* pRpcMsg); -void syncTimeoutFromRpcMsg(const SRpcMsg* pRpcMsg, SyncTimeout* pMsg); -SyncTimeout* syncTimeoutFromRpcMsg2(const SRpcMsg* pRpcMsg); // -cJSON* syncTimeout2Json(const SyncTimeout* pMsg); -char* syncTimeout2Str(const SyncTimeout* pMsg); // - -// for debug ---------------------- -void syncTimeoutPrint(const SyncTimeout* pMsg); -void syncTimeoutPrint2(char* s, const SyncTimeout* pMsg); -void syncTimeoutLog(const SyncTimeout* pMsg); -void syncTimeoutLog2(char* s, const SyncTimeout* pMsg); - -// --------------------------------------------- -typedef struct SyncPing { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - uint32_t dataLen; - char data[]; -} SyncPing; - -SyncPing* syncPingBuild(uint32_t dataLen); -SyncPing* syncPingBuild2(const SRaftId* srcId, const SRaftId* destId, const char* str); -SyncPing* syncPingBuild3(const SRaftId* srcId, const SRaftId* destId); -void syncPingDestroy(SyncPing* pMsg); -void syncPingSerialize(const SyncPing* pMsg, char* buf, uint32_t bufLen); -void syncPingDeserialize(const char* buf, uint32_t len, SyncPing* pMsg); -char* syncPingSerialize2(const SyncPing* pMsg, uint32_t* len); -SyncPing* syncPingDeserialize2(const char* buf, uint32_t len); -void syncPing2RpcMsg(const SyncPing* pMsg, SRpcMsg* pRpcMsg); -void syncPingFromRpcMsg(const SRpcMsg* pRpcMsg, SyncPing* pMsg); -SyncPing* syncPingFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncPing2Json(const SyncPing* pMsg); -char* syncPing2Str(const SyncPing* pMsg); - -// for debug ---------------------- -void syncPingPrint(const SyncPing* pMsg); -void syncPingPrint2(char* s, const SyncPing* pMsg); -void syncPingLog(const SyncPing* pMsg); -void syncPingLog2(char* s, const SyncPing* pMsg); - -// --------------------------------------------- -typedef struct SyncPingReply { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - uint32_t dataLen; - char data[]; -} SyncPingReply; - -SyncPingReply* syncPingReplyBuild(uint32_t dataLen); -SyncPingReply* syncPingReplyBuild2(const SRaftId* srcId, const SRaftId* destId, const char* str); -SyncPingReply* syncPingReplyBuild3(const SRaftId* srcId, const SRaftId* destId); -void syncPingReplyDestroy(SyncPingReply* pMsg); -void syncPingReplySerialize(const SyncPingReply* pMsg, char* buf, uint32_t bufLen); -void syncPingReplyDeserialize(const char* buf, uint32_t len, SyncPingReply* pMsg); -char* syncPingReplySerialize2(const SyncPingReply* pMsg, uint32_t* len); // -SyncPingReply* syncPingReplyDeserialize2(const char* buf, uint32_t len); // -void syncPingReply2RpcMsg(const SyncPingReply* pMsg, SRpcMsg* pRpcMsg); -void syncPingReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncPingReply* pMsg); -SyncPingReply* syncPingReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); // -cJSON* syncPingReply2Json(const SyncPingReply* pMsg); -char* syncPingReply2Str(const SyncPingReply* pMsg); // - -// for debug ---------------------- -void syncPingReplyPrint(const SyncPingReply* pMsg); -void syncPingReplyPrint2(char* s, const SyncPingReply* pMsg); -void syncPingReplyLog(const SyncPingReply* pMsg); -void syncPingReplyLog2(char* s, const SyncPingReply* pMsg); - -// --------------------------------------------- -typedef struct SyncClientRequest { - uint32_t bytes; - uint32_t msgType; - uint32_t originalRpcType; - uint64_t seqNum; - bool isWeak; - uint32_t dataLen; - char data[]; -} SyncClientRequest; - -SyncClientRequest* syncClientRequestBuild(uint32_t dataLen); -SyncClientRequest* syncClientRequestBuild2(const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, bool isWeak); // 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); - -// for debug ---------------------- -void syncClientRequestPrint(const SyncClientRequest* pMsg); -void syncClientRequestPrint2(char* s, const SyncClientRequest* pMsg); -void syncClientRequestLog(const SyncClientRequest* pMsg); -void syncClientRequestLog2(char* s, const SyncClientRequest* pMsg); - -// --------------------------------------------- -typedef struct SyncClientRequestReply { - uint32_t bytes; - uint32_t msgType; - int32_t errCode; - SRaftId leaderHint; -} SyncClientRequestReply; - -// --------------------------------------------- -typedef struct SyncRequestVote { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - SyncTerm term; - SyncIndex lastLogIndex; - SyncTerm lastLogTerm; -} SyncRequestVote; - -SyncRequestVote* syncRequestVoteBuild(); -void syncRequestVoteDestroy(SyncRequestVote* pMsg); -void syncRequestVoteSerialize(const SyncRequestVote* pMsg, char* buf, uint32_t bufLen); -void syncRequestVoteDeserialize(const char* buf, uint32_t len, SyncRequestVote* pMsg); -char* syncRequestVoteSerialize2(const SyncRequestVote* pMsg, uint32_t* len); -SyncRequestVote* syncRequestVoteDeserialize2(const char* buf, uint32_t len); -void syncRequestVote2RpcMsg(const SyncRequestVote* pMsg, SRpcMsg* pRpcMsg); -void syncRequestVoteFromRpcMsg(const SRpcMsg* pRpcMsg, SyncRequestVote* pMsg); -SyncRequestVote* syncRequestVoteFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncRequestVote2Json(const SyncRequestVote* pMsg); -char* syncRequestVote2Str(const SyncRequestVote* pMsg); - -// for debug ---------------------- -void syncRequestVotePrint(const SyncRequestVote* pMsg); -void syncRequestVotePrint2(char* s, const SyncRequestVote* pMsg); -void syncRequestVoteLog(const SyncRequestVote* pMsg); -void syncRequestVoteLog2(char* s, const SyncRequestVote* pMsg); - -// --------------------------------------------- -typedef struct SyncRequestVoteReply { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - SyncTerm term; - bool voteGranted; -} SyncRequestVoteReply; - -SyncRequestVoteReply* syncRequestVoteReplyBuild(); -void syncRequestVoteReplyDestroy(SyncRequestVoteReply* pMsg); -void syncRequestVoteReplySerialize(const SyncRequestVoteReply* pMsg, char* buf, uint32_t bufLen); -void syncRequestVoteReplyDeserialize(const char* buf, uint32_t len, SyncRequestVoteReply* pMsg); -char* syncRequestVoteReplySerialize2(const SyncRequestVoteReply* pMsg, uint32_t* len); -SyncRequestVoteReply* syncRequestVoteReplyDeserialize2(const char* buf, uint32_t len); -void syncRequestVoteReply2RpcMsg(const SyncRequestVoteReply* pMsg, SRpcMsg* pRpcMsg); -void syncRequestVoteReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncRequestVoteReply* pMsg); -SyncRequestVoteReply* syncRequestVoteReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncRequestVoteReply2Json(const SyncRequestVoteReply* pMsg); -char* syncRequestVoteReply2Str(const SyncRequestVoteReply* pMsg); - -// for debug ---------------------- -void syncRequestVoteReplyPrint(const SyncRequestVoteReply* pMsg); -void syncRequestVoteReplyPrint2(char* s, const SyncRequestVoteReply* pMsg); -void syncRequestVoteReplyLog(const SyncRequestVoteReply* pMsg); -void syncRequestVoteReplyLog2(char* s, const SyncRequestVoteReply* pMsg); - -// --------------------------------------------- -typedef struct SyncAppendEntries { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - SyncTerm term; - SyncIndex prevLogIndex; - SyncTerm prevLogTerm; - SyncIndex commitIndex; - uint32_t dataLen; - char data[]; -} SyncAppendEntries; - -SyncAppendEntries* syncAppendEntriesBuild(uint32_t dataLen); -void syncAppendEntriesDestroy(SyncAppendEntries* pMsg); -void syncAppendEntriesSerialize(const SyncAppendEntries* pMsg, char* buf, uint32_t bufLen); -void syncAppendEntriesDeserialize(const char* buf, uint32_t len, SyncAppendEntries* pMsg); -char* syncAppendEntriesSerialize2(const SyncAppendEntries* pMsg, uint32_t* len); -SyncAppendEntries* syncAppendEntriesDeserialize2(const char* buf, uint32_t len); -void syncAppendEntries2RpcMsg(const SyncAppendEntries* pMsg, SRpcMsg* pRpcMsg); -void syncAppendEntriesFromRpcMsg(const SRpcMsg* pRpcMsg, SyncAppendEntries* pMsg); -SyncAppendEntries* syncAppendEntriesFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncAppendEntries2Json(const SyncAppendEntries* pMsg); -char* syncAppendEntries2Str(const SyncAppendEntries* pMsg); - -// for debug ---------------------- -void syncAppendEntriesPrint(const SyncAppendEntries* pMsg); -void syncAppendEntriesPrint2(char* s, const SyncAppendEntries* pMsg); -void syncAppendEntriesLog(const SyncAppendEntries* pMsg); -void syncAppendEntriesLog2(char* s, const SyncAppendEntries* pMsg); - -// --------------------------------------------- -typedef struct SyncAppendEntriesReply { - uint32_t bytes; - uint32_t msgType; - SRaftId srcId; - SRaftId destId; - // private data - SyncTerm term; - bool success; - SyncIndex matchIndex; -} SyncAppendEntriesReply; - -SyncAppendEntriesReply* syncAppendEntriesReplyBuild(); -void syncAppendEntriesReplyDestroy(SyncAppendEntriesReply* pMsg); -void syncAppendEntriesReplySerialize(const SyncAppendEntriesReply* pMsg, char* buf, uint32_t bufLen); -void syncAppendEntriesReplyDeserialize(const char* buf, uint32_t len, SyncAppendEntriesReply* pMsg); -char* syncAppendEntriesReplySerialize2(const SyncAppendEntriesReply* pMsg, uint32_t* len); -SyncAppendEntriesReply* syncAppendEntriesReplyDeserialize2(const char* buf, uint32_t len); -void syncAppendEntriesReply2RpcMsg(const SyncAppendEntriesReply* pMsg, SRpcMsg* pRpcMsg); -void syncAppendEntriesReplyFromRpcMsg(const SRpcMsg* pRpcMsg, SyncAppendEntriesReply* pMsg); -SyncAppendEntriesReply* syncAppendEntriesReplyFromRpcMsg2(const SRpcMsg* pRpcMsg); -cJSON* syncAppendEntriesReply2Json(const SyncAppendEntriesReply* pMsg); -char* syncAppendEntriesReply2Str(const SyncAppendEntriesReply* pMsg); - -// for debug ---------------------- -void syncAppendEntriesReplyPrint(const SyncAppendEntriesReply* pMsg); -void syncAppendEntriesReplyPrint2(char* s, const SyncAppendEntriesReply* pMsg); -void syncAppendEntriesReplyLog(const SyncAppendEntriesReply* pMsg); -void syncAppendEntriesReplyLog2(char* s, const SyncAppendEntriesReply* pMsg); #ifdef __cplusplus } diff --git a/source/libs/sync/inc/syncRaftCfg.h b/source/libs/sync/inc/syncRaftCfg.h new file mode 100644 index 0000000000..bfc64cb7b6 --- /dev/null +++ b/source/libs/sync/inc/syncRaftCfg.h @@ -0,0 +1,65 @@ +/* + * 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_RAFT_CFG_H +#define _TD_LIBS_SYNC_RAFT_CFG_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include +#include +#include +#include "cJSON.h" +#include "syncInt.h" +#include "taosdef.h" + +typedef struct SRaftCfg { + SSyncCfg cfg; + TdFilePtr pFile; + char path[TSDB_FILENAME_LEN * 2]; +} SRaftCfg; + +SRaftCfg *raftCfgOpen(const char *path); +int32_t raftCfgClose(SRaftCfg *pRaftCfg); +int32_t raftCfgPersist(SRaftCfg *pRaftCfg); + +cJSON * syncCfg2Json(SSyncCfg *pSyncCfg); +char * syncCfg2Str(SSyncCfg *pSyncCfg); +int32_t syncCfgFromJson(const cJSON *pRoot, SSyncCfg *pSyncCfg); +int32_t syncCfgFromStr(const char *s, SSyncCfg *pSyncCfg); + +cJSON *raftCfg2Json(SRaftCfg *pRaftCfg); +char * raftCfg2Str(SRaftCfg *pRaftCfg); + +int32_t syncCfgCreateFile(SSyncCfg *pCfg, const char *path); + +// for debug ---------------------- +void syncCfgPrint(SSyncCfg *pCfg); +void syncCfgPrint2(char *s, SSyncCfg *pCfg); +void syncCfgLog(SSyncCfg *pCfg); +void syncCfgLog2(char *s, SSyncCfg *pCfg); + +void raftCfgPrint(SRaftCfg *pCfg); +void raftCfgPrint2(char *s, SRaftCfg *pCfg); +void raftCfgLog(SRaftCfg *pCfg); +void raftCfgLog2(char *s, SRaftCfg *pCfg); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_LIBS_SYNC_RAFT_CFG_H*/ diff --git a/source/libs/sync/inc/syncRaftEntry.h b/source/libs/sync/inc/syncRaftEntry.h index d4e63e9676..b130186516 100644 --- a/source/libs/sync/inc/syncRaftEntry.h +++ b/source/libs/sync/inc/syncRaftEntry.h @@ -27,29 +27,22 @@ extern "C" { #include "syncMessage.h" #include "taosdef.h" -typedef enum EntryType { - SYNC_RAFT_ENTRY_NOOP = 0, - SYNC_RAFT_ENTRY_DATA = 1, - SYNC_RAFT_ENTRY_CONFIG = 2, -} EntryType; - typedef struct SSyncRaftEntry { uint32_t bytes; - uint32_t msgType; - uint32_t originalRpcType; + uint32_t msgType; // SyncClientRequest msgType + uint32_t originalRpcType; // user RpcMsg msgType uint64_t seqNum; bool isWeak; SyncTerm term; SyncIndex index; - EntryType entryType; - uint32_t dataLen; - char data[]; + uint32_t dataLen; // user RpcMsg.contLen + char data[]; // user 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, EntryType entryType); -SSyncRaftEntry* syncEntryBuildNoop(SyncTerm term, SyncIndex index); +SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index); +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 diff --git a/source/libs/sync/inc/syncRaftStore.h b/source/libs/sync/inc/syncRaftStore.h index 355a08ac84..e0cbcf0744 100644 --- a/source/libs/sync/inc/syncRaftStore.h +++ b/source/libs/sync/inc/syncRaftStore.h @@ -28,7 +28,7 @@ extern "C" { #include "taosdef.h" #define RAFT_STORE_BLOCK_SIZE 512 -#define RAFT_STORE_PATH_LEN 128 +#define RAFT_STORE_PATH_LEN (TSDB_FILENAME_LEN * 2) typedef struct SRaftStore { SyncTerm currentTerm; diff --git a/source/libs/sync/inc/syncRespMgr.h b/source/libs/sync/inc/syncRespMgr.h new file mode 100644 index 0000000000..e37c0bb625 --- /dev/null +++ b/source/libs/sync/inc/syncRespMgr.h @@ -0,0 +1,55 @@ +/* + * 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_RESP_MGR_H +#define _TD_LIBS_SYNC_RESP_MGR_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include +#include +#include +#include "syncInt.h" +#include "taosdef.h" + +typedef struct SRespStub { + SRpcMsg rpcMsg; + int64_t createTime; +} SRespStub; + +typedef struct SSyncRespMgr { + SHashObj * pRespHash; + int64_t ttl; + void * data; + TdThreadMutex mutex; + uint64_t seqNum; +} SSyncRespMgr; + +SSyncRespMgr *syncRespMgrCreate(void *data, int64_t ttl); +void syncRespMgrDestroy(SSyncRespMgr *pObj); +int64_t syncRespMgrAdd(SSyncRespMgr *pObj, SRespStub *pStub); +int32_t syncRespMgrDel(SSyncRespMgr *pObj, uint64_t index); +int32_t syncRespMgrGet(SSyncRespMgr *pObj, uint64_t index, SRespStub *pStub); +int32_t syncRespMgrGetAndDel(SSyncRespMgr *pObj, uint64_t index, SRespStub *pStub); +void syncRespClean(SSyncRespMgr *pObj); +void syncRespCleanByTTL(SSyncRespMgr *pObj, int64_t ttl); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_LIBS_SYNC_RESP_MGR_H*/ diff --git a/source/libs/sync/inc/syncSnapshot.h b/source/libs/sync/inc/syncSnapshot.h index 611f33a0f2..fd2119ce65 100644 --- a/source/libs/sync/inc/syncSnapshot.h +++ b/source/libs/sync/inc/syncSnapshot.h @@ -27,7 +27,6 @@ extern "C" { #include "taosdef.h" int32_t takeSnapshot(SSyncFSM *pFsm, SSnapshot *pSnapshot); - int32_t restoreSnapshot(SSyncFSM *pFsm, SSnapshot *pSnapshot); #ifdef __cplusplus diff --git a/source/libs/sync/inc/syncUtil.h b/source/libs/sync/inc/syncUtil.h index a4bb11afc6..83f31c5dac 100644 --- a/source/libs/sync/inc/syncUtil.h +++ b/source/libs/sync/inc/syncUtil.h @@ -55,6 +55,8 @@ char* syncUtilprintBin(char* ptr, uint32_t len); char* syncUtilprintBin2(char* ptr, uint32_t len); SyncIndex syncUtilMinIndex(SyncIndex a, SyncIndex b); SyncIndex syncUtilMaxIndex(SyncIndex a, SyncIndex b); +void syncUtilMsgHtoN(void* msg); +void syncUtilMsgNtoH(void* msg); #ifdef __cplusplus } diff --git a/source/libs/sync/src/syncAppendEntries.c b/source/libs/sync/src/syncAppendEntries.c index 2e9cd63de6..6623ed1caa 100644 --- a/source/libs/sync/src/syncAppendEntries.c +++ b/source/libs/sync/src/syncAppendEntries.c @@ -125,7 +125,7 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { "logOK:%d", pMsg->term, ths->pRaftStore->currentTerm, ths->state, logOK); - SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(ths->vgId); pReply->srcId = ths->myRaftId; pReply->destId = pMsg->srcId; pReply->term = ths->pRaftStore->currentTerm; @@ -199,13 +199,20 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { SSyncRaftEntry* pRollBackEntry = logStoreGetEntry(ths->pLogStore, index); assert(pRollBackEntry != NULL); - // maybe is a NOOP ENTRY - // assert(pRollBackEntry->entryType == SYNC_RAFT_ENTRY_DATA); + if (pRollBackEntry->msgType != TDMT_VND_SYNC_NOOP) { + SRpcMsg rpcMsg; + syncEntry2OriginalRpc(pRollBackEntry, &rpcMsg); + + SFsmCbMeta cbMeta; + cbMeta.index = pRollBackEntry->index; + cbMeta.isWeak = pRollBackEntry->isWeak; + cbMeta.code = 0; + cbMeta.state = ths->state; + cbMeta.seqNum = pRollBackEntry->seqNum; + ths->pFsm->FpRollBackCb(ths->pFsm, &rpcMsg, cbMeta); + rpcFreeCont(rpcMsg.pCont); + } - SRpcMsg rpcMsg; - syncEntry2OriginalRpc(pRollBackEntry, &rpcMsg); - ths->pFsm->FpRollBackCb(ths->pFsm, &rpcMsg, pRollBackEntry->index, pRollBackEntry->isWeak, 0, ths->state); - rpcFreeCont(rpcMsg.pCont); syncEntryDestory(pRollBackEntry); } } @@ -220,8 +227,14 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { SRpcMsg rpcMsg; syncEntry2OriginalRpc(pAppendEntry, &rpcMsg); if (ths->pFsm != NULL) { - if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, pAppendEntry->index, pAppendEntry->isWeak, 2, ths->state); + if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pAppendEntry->index; + cbMeta.isWeak = pAppendEntry->isWeak; + cbMeta.code = 2; + cbMeta.state = ths->state; + cbMeta.seqNum = pAppendEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); } } rpcFreeCont(rpcMsg.pCont); @@ -245,8 +258,14 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { SRpcMsg rpcMsg; syncEntry2OriginalRpc(pAppendEntry, &rpcMsg); if (ths->pFsm != NULL) { - if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, pAppendEntry->index, pAppendEntry->isWeak, 3, ths->state); + if (ths->pFsm->FpPreCommitCb != NULL && pAppendEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pAppendEntry->index; + cbMeta.isWeak = pAppendEntry->isWeak; + cbMeta.code = 3; + cbMeta.state = ths->state; + cbMeta.seqNum = pAppendEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); } } rpcFreeCont(rpcMsg.pCont); @@ -261,7 +280,7 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { assert(0); } - SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply* pReply = syncAppendEntriesReplyBuild(ths->vgId); pReply->srcId = ths->myRaftId; pReply->destId = pMsg->srcId; pReply->term = ths->pRaftStore->currentTerm; @@ -301,8 +320,14 @@ int32_t syncNodeOnAppendEntriesCb(SSyncNode* ths, SyncAppendEntries* pMsg) { SRpcMsg rpcMsg; syncEntry2OriginalRpc(pEntry, &rpcMsg); - if (ths->pFsm->FpCommitCb != NULL && pEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - ths->pFsm->FpCommitCb(ths->pFsm, &rpcMsg, pEntry->index, pEntry->isWeak, 0, ths->state); + if (ths->pFsm->FpCommitCb != NULL && pEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pEntry->index; + cbMeta.isWeak = pEntry->isWeak; + cbMeta.code = 0; + cbMeta.state = ths->state; + cbMeta.seqNum = pEntry->seqNum; + ths->pFsm->FpCommitCb(ths->pFsm, &rpcMsg, cbMeta); } rpcFreeCont(rpcMsg.pCont); diff --git a/source/libs/sync/src/syncAppendEntriesReply.c b/source/libs/sync/src/syncAppendEntriesReply.c index 790ac7f8e1..77d85e2915 100644 --- a/source/libs/sync/src/syncAppendEntriesReply.c +++ b/source/libs/sync/src/syncAppendEntriesReply.c @@ -56,6 +56,15 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p // syncNodeUpdateTerm(ths, pMsg->term); // } + if (pMsg->term > ths->pRaftStore->currentTerm) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "syncNodeOnAppendEntriesReplyCb error term, receive:%lu current:%lu", pMsg->term, + ths->pRaftStore->currentTerm); + syncNodeLog2(logBuf, ths); + sError("%s", logBuf); + return ret; + } + assert(pMsg->term == ths->pRaftStore->currentTerm); if (pMsg->success) { diff --git a/source/libs/sync/src/syncCommit.c b/source/libs/sync/src/syncCommit.c index 6023c00afa..8b54b0a090 100644 --- a/source/libs/sync/src/syncCommit.c +++ b/source/libs/sync/src/syncCommit.c @@ -97,8 +97,14 @@ void syncMaybeAdvanceCommitIndex(SSyncNode* pSyncNode) { SRpcMsg rpcMsg; syncEntry2OriginalRpc(pEntry, &rpcMsg); - if (pSyncNode->pFsm->FpCommitCb != NULL && pEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - pSyncNode->pFsm->FpCommitCb(pSyncNode->pFsm, &rpcMsg, pEntry->index, pEntry->isWeak, 0, pSyncNode->state); + if (pSyncNode->pFsm->FpCommitCb != NULL && pEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pEntry->index; + cbMeta.isWeak = pEntry->isWeak; + cbMeta.code = 0; + cbMeta.state = pSyncNode->state; + cbMeta.seqNum = pEntry->seqNum; + pSyncNode->pFsm->FpCommitCb(pSyncNode->pFsm, &rpcMsg, cbMeta); } rpcFreeCont(rpcMsg.pCont); diff --git a/source/libs/sync/src/syncElection.c b/source/libs/sync/src/syncElection.c index 6ae70689ef..5101344b84 100644 --- a/source/libs/sync/src/syncElection.c +++ b/source/libs/sync/src/syncElection.c @@ -35,7 +35,7 @@ int32_t syncNodeRequestVotePeers(SSyncNode* pSyncNode) { int32_t ret = 0; for (int i = 0; i < pSyncNode->peersNum; ++i) { - SyncRequestVote* pMsg = syncRequestVoteBuild(); + SyncRequestVote* pMsg = syncRequestVoteBuild(pSyncNode->vgId); pMsg->srcId = pSyncNode->myRaftId; pMsg->destId = pSyncNode->peersId[i]; pMsg->term = pSyncNode->pRaftStore->currentTerm; diff --git a/source/libs/sync/src/syncEnv.c b/source/libs/sync/src/syncEnv.c index 726ebdae07..4213d7b45d 100644 --- a/source/libs/sync/src/syncEnv.c +++ b/source/libs/sync/src/syncEnv.c @@ -27,10 +27,13 @@ static void syncEnvTick(void *param, void *tmrId); // -------------------------------- int32_t syncEnvStart() { + sDebugFlag = 207; + int32_t ret = 0; taosSeedRand(taosGetTimestampSec()); gSyncEnv = doSyncEnvStart(gSyncEnv); assert(gSyncEnv != NULL); + sTrace("syncEnvStart ok!"); return ret; } diff --git a/source/libs/sync/src/syncIO.c b/source/libs/sync/src/syncIO.c index 717b1cc756..b1d7af3254 100644 --- a/source/libs/sync/src/syncIO.c +++ b/source/libs/sync/src/syncIO.c @@ -29,7 +29,7 @@ static int32_t syncIODestroy(SSyncIO *io); static int32_t syncIOStartInternal(SSyncIO *io); static int32_t syncIOStopInternal(SSyncIO *io); -static void *syncIOConsumerFunc(void *param); +static void * syncIOConsumerFunc(void *param); static void syncIOProcessRequest(void *pParent, SRpcMsg *pMsg, SEpSet *pEpSet); static void syncIOProcessReply(void *pParent, SRpcMsg *pMsg, SEpSet *pEpSet); static int32_t syncIOAuth(void *parent, char *meterId, char *spi, char *encrypt, char *secret, char *ckey); @@ -70,9 +70,15 @@ int32_t syncIOSendMsg(void *clientRpc, const SEpSet *pEpSet, SRpcMsg *pMsg) { assert(pEpSet->numOfEps == 1); int32_t ret = 0; - char logBuf[256]; - snprintf(logBuf, sizeof(logBuf), "==syncIOSendMsg== %s:%d", pEpSet->eps[0].fqdn, pEpSet->eps[0].port); - syncRpcMsgPrint2(logBuf, pMsg); + { + syncUtilMsgNtoH(pMsg->pCont); + + char logBuf[256]; + snprintf(logBuf, sizeof(logBuf), "==syncIOSendMsg== %s:%d", pEpSet->eps[0].fqdn, pEpSet->eps[0].port); + syncRpcMsgLog2(logBuf, pMsg); + + syncUtilMsgHtoN(pMsg->pCont); + } pMsg->handle = NULL; pMsg->noResp = 1; @@ -83,7 +89,7 @@ int32_t syncIOSendMsg(void *clientRpc, const SEpSet *pEpSet, SRpcMsg *pMsg) { int32_t syncIOEqMsg(void *queue, SRpcMsg *pMsg) { int32_t ret = 0; char logBuf[128]; - syncRpcMsgPrint2((char *)"==syncIOEqMsg==", pMsg); + syncRpcMsgLog2((char *)"==syncIOEqMsg==", pMsg); SRpcMsg *pTemp; pTemp = taosAllocateQitem(sizeof(SRpcMsg)); @@ -235,9 +241,9 @@ static int32_t syncIOStopInternal(SSyncIO *io) { } static void *syncIOConsumerFunc(void *param) { - SSyncIO *io = param; + SSyncIO * io = param; STaosQall *qall; - SRpcMsg *pRpcMsg, rpcMsg; + SRpcMsg * pRpcMsg, rpcMsg; qall = taosAllocateQall(); while (1) { @@ -252,7 +258,7 @@ static void *syncIOConsumerFunc(void *param) { syncRpcMsgLog2((char *)"==syncIOConsumerFunc==", pRpcMsg); // use switch case instead of if else - if (pRpcMsg->msgType == SYNC_PING) { + if (pRpcMsg->msgType == TDMT_VND_SYNC_PING) { if (io->FpOnSyncPing != NULL) { SyncPing *pSyncMsg = syncPingFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -260,7 +266,7 @@ static void *syncIOConsumerFunc(void *param) { syncPingDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_PING_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_PING_REPLY) { if (io->FpOnSyncPingReply != NULL) { SyncPingReply *pSyncMsg = syncPingReplyFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -268,7 +274,7 @@ static void *syncIOConsumerFunc(void *param) { syncPingReplyDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_CLIENT_REQUEST) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_CLIENT_REQUEST) { if (io->FpOnSyncClientRequest != NULL) { SyncClientRequest *pSyncMsg = syncClientRequestFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -276,7 +282,7 @@ static void *syncIOConsumerFunc(void *param) { syncClientRequestDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_REQUEST_VOTE) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_REQUEST_VOTE) { if (io->FpOnSyncRequestVote != NULL) { SyncRequestVote *pSyncMsg = syncRequestVoteFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -284,7 +290,7 @@ static void *syncIOConsumerFunc(void *param) { syncRequestVoteDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_REQUEST_VOTE_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_REQUEST_VOTE_REPLY) { if (io->FpOnSyncRequestVoteReply != NULL) { SyncRequestVoteReply *pSyncMsg = syncRequestVoteReplyFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -292,7 +298,7 @@ static void *syncIOConsumerFunc(void *param) { syncRequestVoteReplyDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_APPEND_ENTRIES) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_APPEND_ENTRIES) { if (io->FpOnSyncAppendEntries != NULL) { SyncAppendEntries *pSyncMsg = syncAppendEntriesFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -300,7 +306,7 @@ static void *syncIOConsumerFunc(void *param) { syncAppendEntriesDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_APPEND_ENTRIES_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_APPEND_ENTRIES_REPLY) { if (io->FpOnSyncAppendEntriesReply != NULL) { SyncAppendEntriesReply *pSyncMsg = syncAppendEntriesReplyFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -308,7 +314,7 @@ static void *syncIOConsumerFunc(void *param) { syncAppendEntriesReplyDestroy(pSyncMsg); } - } else if (pRpcMsg->msgType == SYNC_TIMEOUT) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_TIMEOUT) { if (io->FpOnSyncTimeout != NULL) { SyncTimeout *pSyncMsg = syncTimeoutFromRpcMsg2(pRpcMsg); assert(pSyncMsg != NULL); @@ -336,7 +342,7 @@ static void *syncIOConsumerFunc(void *param) { rpcMsg.handle = pRpcMsg->handle; rpcMsg.code = 0; - syncRpcMsgPrint2((char *)"syncIOConsumerFunc rpcSendResponse --> ", &rpcMsg); + syncRpcMsgLog2((char *)"syncIOConsumerFunc rpcSendResponse --> ", &rpcMsg); rpcSendResponse(&rpcMsg); } */ @@ -350,7 +356,9 @@ static void *syncIOConsumerFunc(void *param) { } static void syncIOProcessRequest(void *pParent, SRpcMsg *pMsg, SEpSet *pEpSet) { - syncRpcMsgPrint2((char *)"==syncIOProcessRequest==", pMsg); + syncUtilMsgNtoH(pMsg->pCont); + + syncRpcMsgLog2((char *)"==syncIOProcessRequest==", pMsg); SSyncIO *io = pParent; SRpcMsg *pTemp; pTemp = taosAllocateQitem(sizeof(SRpcMsg)); @@ -359,10 +367,10 @@ static void syncIOProcessRequest(void *pParent, SRpcMsg *pMsg, SEpSet *pEpSet) { } static void syncIOProcessReply(void *pParent, SRpcMsg *pMsg, SEpSet *pEpSet) { - if (pMsg->msgType == SYNC_RESPONSE) { + if (pMsg->msgType == TDMT_VND_SYNC_COMMON_RESPONSE) { sTrace("==syncIOProcessReply=="); } else { - syncRpcMsgPrint2((char *)"==syncIOProcessReply==", pMsg); + syncRpcMsgLog2((char *)"==syncIOProcessReply==", pMsg); } rpcFreeCont(pMsg->pCont); } @@ -408,14 +416,14 @@ static void syncIOTickQ(void *param, void *tmrId) { srcId.vgId = -1; destId.addr = syncUtilAddr2U64(io->myAddr.eps[0].fqdn, io->myAddr.eps[0].port); destId.vgId = -1; - SyncPingReply *pMsg = syncPingReplyBuild2(&srcId, &destId, "syncIOTickQ"); + SyncPingReply *pMsg = syncPingReplyBuild2(&srcId, &destId, -1, "syncIOTickQ"); SRpcMsg rpcMsg; syncPingReply2RpcMsg(pMsg, &rpcMsg); SRpcMsg *pTemp; pTemp = taosAllocateQitem(sizeof(SRpcMsg)); memcpy(pTemp, &rpcMsg, sizeof(SRpcMsg)); - syncRpcMsgPrint2((char *)"==syncIOTickQ==", &rpcMsg); + syncRpcMsgLog2((char *)"==syncIOTickQ==", &rpcMsg); taosWriteQitem(io->pMsgQ, pTemp); syncPingReplyDestroy(pMsg); @@ -430,12 +438,12 @@ static void syncIOTickPing(void *param, void *tmrId) { srcId.vgId = -1; destId.addr = syncUtilAddr2U64(io->myAddr.eps[0].fqdn, io->myAddr.eps[0].port); destId.vgId = -1; - SyncPing *pMsg = syncPingBuild2(&srcId, &destId, "syncIOTickPing"); + SyncPing *pMsg = syncPingBuild2(&srcId, &destId, -1, "syncIOTickPing"); // SyncPing *pMsg = syncPingBuild3(&srcId, &destId); SRpcMsg rpcMsg; syncPing2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"==syncIOTickPing==", &rpcMsg); + syncRpcMsgLog2((char *)"==syncIOTickPing==", &rpcMsg); rpcSendRequest(io->clientRpc, &io->myAddr, &rpcMsg, NULL); syncPingDestroy(pMsg); diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index 78e454309a..6d51cee14e 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -23,11 +23,13 @@ #include "syncIndexMgr.h" #include "syncInt.h" #include "syncMessage.h" +#include "syncRaftCfg.h" #include "syncRaftLog.h" #include "syncRaftStore.h" #include "syncReplication.h" #include "syncRequestVote.h" #include "syncRequestVoteReply.h" +#include "syncRespMgr.h" #include "syncTimeout.h" #include "syncUtil.h" #include "syncVoteMgr.h" @@ -44,9 +46,9 @@ static int32_t syncNodeEqNoop(SSyncNode* ths); static int32_t syncNodeAppendNoop(SSyncNode* ths); // process message ---- -static int32_t syncNodeOnPingCb(SSyncNode* ths, SyncPing* pMsg); -static int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg); -static int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg); +int32_t syncNodeOnPingCb(SSyncNode* ths, SyncPing* pMsg); +int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg); +int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg); // life cycle static void syncFreeNode(void* param); @@ -76,10 +78,12 @@ void syncCleanUp() { } } -int64_t syncStart(const SSyncInfo* pSyncInfo) { +int64_t syncOpen(const SSyncInfo* pSyncInfo) { SSyncNode* pSyncNode = syncNodeOpen(pSyncInfo); assert(pSyncNode != NULL); + syncNodeLog2("syncNodeOpen open success", pSyncNode); + pSyncNode->rid = taosAddRef(tsNodeRefId, pSyncNode); if (pSyncNode->rid < 0) { syncFreeNode(pSyncNode); @@ -89,6 +93,16 @@ int64_t syncStart(const SSyncInfo* pSyncInfo) { return pSyncNode->rid; } +void syncStart(int64_t rid) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return; + } + syncNodeStart(pSyncNode); + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + void syncStop(int64_t rid) { SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); if (pSyncNode == NULL) { @@ -105,11 +119,6 @@ int32_t syncReconfig(int64_t rid, const SSyncCfg* pSyncCfg) { return ret; } -int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak) { - int32_t ret = syncPropose2(rid, pMsg, isWeak, 0); - return ret; -} - int32_t syncForwardToPeer(int64_t rid, const SRpcMsg* pMsg, bool isWeak) { int32_t ret = syncPropose(rid, pMsg, isWeak); return ret; @@ -121,28 +130,163 @@ ESyncState syncGetMyRole(int64_t rid) { return TAOS_SYNC_STATE_ERROR; } assert(rid == pSyncNode->rid); - return pSyncNode->state; + ESyncState state = pSyncNode->state; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); + return state; } -int32_t syncPropose2(int64_t rid, const SRpcMsg* pMsg, bool isWeak, uint64_t seqNum) { - int32_t ret = 0; +const char* syncGetMyRoleStr(int64_t rid) { + const char* s = syncUtilState2String(syncGetMyRole(rid)); + return s; +} + +SyncTerm syncGetMyTerm(int64_t rid) { SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); if (pSyncNode == NULL) { - return -1; + return TAOS_SYNC_STATE_ERROR; + } + assert(rid == pSyncNode->rid); + SyncTerm term = pSyncNode->pRaftStore->currentTerm; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); + return term; +} + +int32_t syncGetRespRpc(int64_t rid, uint64_t index, SRpcMsg* msg) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return TAOS_SYNC_STATE_ERROR; + } + assert(rid == pSyncNode->rid); + + SRespStub stub; + int32_t ret = syncRespMgrGet(pSyncNode->pSyncRespMgr, index, &stub); + if (ret == 1) { + memcpy(msg, &(stub.rpcMsg), sizeof(SRpcMsg)); + } + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); + return ret; +} + +int32_t syncGetAndDelRespRpc(int64_t rid, uint64_t index, SRpcMsg* msg) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return TAOS_SYNC_STATE_ERROR; + } + assert(rid == pSyncNode->rid); + + SRespStub stub; + int32_t ret = syncRespMgrGetAndDel(pSyncNode->pSyncRespMgr, index, &stub); + if (ret == 1) { + memcpy(msg, &(stub.rpcMsg), sizeof(SRpcMsg)); + } + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); + return ret; +} + +void syncSetQ(int64_t rid, void* queue) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + sTrace("syncSetQ get pSyncNode is NULL, rid:%ld", rid); + return; + } + assert(rid == pSyncNode->rid); + pSyncNode->queue = queue; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + +void syncSetRpc(int64_t rid, void* rpcHandle) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + sTrace("syncSetRpc get pSyncNode is NULL, rid:%ld", rid); + return; + } + assert(rid == pSyncNode->rid); + pSyncNode->rpcClient = rpcHandle; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + +char* sync2SimpleStr(int64_t rid) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + sTrace("syncSetRpc get pSyncNode is NULL, rid:%ld", rid); + return NULL; + } + assert(rid == pSyncNode->rid); + char* s = syncNode2SimpleStr(pSyncNode); + taosReleaseRef(tsNodeRefId, pSyncNode->rid); + + return s; +} + +void setPingTimerMS(int64_t rid, int32_t pingTimerMS) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return; + } + assert(rid == pSyncNode->rid); + pSyncNode->pingBaseLine = pingTimerMS; + pSyncNode->pingTimerMS = pingTimerMS; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + +void setElectTimerMS(int64_t rid, int32_t electTimerMS) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return; + } + assert(rid == pSyncNode->rid); + pSyncNode->electBaseLine = electTimerMS; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + +void setHeartbeatTimerMS(int64_t rid, int32_t hbTimerMS) { + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return; + } + assert(rid == pSyncNode->rid); + pSyncNode->hbBaseLine = hbTimerMS; + pSyncNode->heartbeatTimerMS = hbTimerMS; + + taosReleaseRef(tsNodeRefId, pSyncNode->rid); +} + +int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak) { + int32_t ret = TAOS_SYNC_PROPOSE_SUCCESS; + SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid); + if (pSyncNode == NULL) { + return TAOS_SYNC_PROPOSE_OTHER_ERROR; } assert(rid == pSyncNode->rid); if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) { - SyncClientRequest* pSyncMsg = syncClientRequestBuild2(pMsg, seqNum, isWeak); + SRespStub stub; + stub.createTime = taosGetTimestampMs(); + stub.rpcMsg = *pMsg; + uint64_t seqNum = syncRespMgrAdd(pSyncNode->pSyncRespMgr, &stub); + + SyncClientRequest* pSyncMsg = syncClientRequestBuild2(pMsg, seqNum, isWeak, pSyncNode->vgId); SRpcMsg rpcMsg; syncClientRequest2RpcMsg(pSyncMsg, &rpcMsg); - pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + if (pSyncNode->FpEqMsg != NULL) { + pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + } else { + sTrace("syncPropose pSyncNode->FpEqMsg is NULL"); + } syncClientRequestDestroy(pSyncMsg); - ret = 0; + ret = TAOS_SYNC_PROPOSE_SUCCESS; } else { sTrace("syncPropose not leader, %s", syncUtilState2String(pSyncNode->state)); - ret = -1; // todo : need define err code !! + ret = TAOS_SYNC_PROPOSE_NOT_LEADER; } taosReleaseRef(tsNodeRefId, pSyncNode->rid); @@ -155,49 +299,62 @@ SSyncNode* syncNodeOpen(const SSyncInfo* pSyncInfo) { assert(pSyncNode != NULL); memset(pSyncNode, 0, sizeof(SSyncNode)); - if (taosMkDir(pSyncInfo->path) != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - sError("failed to create dir:%s since %s", pSyncInfo->path, terrstr()); - return NULL; + int32_t ret = 0; + if (!taosDirExist((char*)(pSyncInfo->path))) { + if (taosMkDir(pSyncInfo->path) != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + sError("failed to create dir:%s since %s", pSyncInfo->path, terrstr()); + return NULL; + } + + // create raft config file + snprintf(pSyncNode->configPath, sizeof(pSyncNode->configPath), "%s/raft_config.json", pSyncInfo->path); + ret = syncCfgCreateFile((SSyncCfg*)&(pSyncInfo->syncCfg), pSyncNode->configPath); + assert(ret == 0); } // init by SSyncInfo pSyncNode->vgId = pSyncInfo->vgId; - pSyncNode->syncCfg = pSyncInfo->syncCfg; memcpy(pSyncNode->path, pSyncInfo->path, sizeof(pSyncNode->path)); snprintf(pSyncNode->raftStorePath, sizeof(pSyncNode->raftStorePath), "%s/raft_store.json", pSyncInfo->path); + snprintf(pSyncNode->configPath, sizeof(pSyncNode->configPath), "%s/raft_config.json", pSyncInfo->path); + pSyncNode->pWal = pSyncInfo->pWal; pSyncNode->rpcClient = pSyncInfo->rpcClient; pSyncNode->FpSendMsg = pSyncInfo->FpSendMsg; pSyncNode->queue = pSyncInfo->queue; pSyncNode->FpEqMsg = pSyncInfo->FpEqMsg; + // init raft config + pSyncNode->pRaftCfg = raftCfgOpen(pSyncNode->configPath); + assert(pSyncNode->pRaftCfg != NULL); + // init internal - pSyncNode->myNodeInfo = pSyncInfo->syncCfg.nodeInfo[pSyncInfo->syncCfg.myIndex]; - syncUtilnodeInfo2raftId(&pSyncNode->myNodeInfo, pSyncInfo->vgId, &pSyncNode->myRaftId); + pSyncNode->myNodeInfo = pSyncNode->pRaftCfg->cfg.nodeInfo[pSyncNode->pRaftCfg->cfg.myIndex]; + syncUtilnodeInfo2raftId(&pSyncNode->myNodeInfo, pSyncNode->vgId, &pSyncNode->myRaftId); // init peersNum, peers, peersId - pSyncNode->peersNum = pSyncInfo->syncCfg.replicaNum - 1; + pSyncNode->peersNum = pSyncNode->pRaftCfg->cfg.replicaNum - 1; int j = 0; - for (int i = 0; i < pSyncInfo->syncCfg.replicaNum; ++i) { - if (i != pSyncInfo->syncCfg.myIndex) { - pSyncNode->peersNodeInfo[j] = pSyncInfo->syncCfg.nodeInfo[i]; + for (int i = 0; i < pSyncNode->pRaftCfg->cfg.replicaNum; ++i) { + if (i != pSyncNode->pRaftCfg->cfg.myIndex) { + pSyncNode->peersNodeInfo[j] = pSyncNode->pRaftCfg->cfg.nodeInfo[i]; j++; } } for (int i = 0; i < pSyncNode->peersNum; ++i) { - syncUtilnodeInfo2raftId(&pSyncNode->peersNodeInfo[i], pSyncInfo->vgId, &pSyncNode->peersId[i]); + syncUtilnodeInfo2raftId(&pSyncNode->peersNodeInfo[i], pSyncNode->vgId, &pSyncNode->peersId[i]); } // init replicaNum, replicasId - pSyncNode->replicaNum = pSyncInfo->syncCfg.replicaNum; - for (int i = 0; i < pSyncInfo->syncCfg.replicaNum; ++i) { - syncUtilnodeInfo2raftId(&pSyncInfo->syncCfg.nodeInfo[i], pSyncInfo->vgId, &pSyncNode->replicasId[i]); + pSyncNode->replicaNum = pSyncNode->pRaftCfg->cfg.replicaNum; + for (int i = 0; i < pSyncNode->pRaftCfg->cfg.replicaNum; ++i) { + syncUtilnodeInfo2raftId(&pSyncNode->pRaftCfg->cfg.nodeInfo[i], pSyncNode->vgId, &pSyncNode->replicasId[i]); } // init raft algorithm pSyncNode->pFsm = pSyncInfo->pFsm; - pSyncNode->quorum = syncUtilQuorum(pSyncInfo->syncCfg.replicaNum); + pSyncNode->quorum = syncUtilQuorum(pSyncNode->pRaftCfg->cfg.replicaNum); pSyncNode->leaderCache = EMPTY_RAFT_ID; // init life cycle outside @@ -287,10 +444,25 @@ SSyncNode* syncNodeOpen(const SSyncInfo* pSyncInfo) { pSyncNode->FpOnAppendEntriesReply = syncNodeOnAppendEntriesReplyCb; pSyncNode->FpOnTimeout = syncNodeOnTimeoutCb; + // tools + pSyncNode->pSyncRespMgr = syncRespMgrCreate(NULL, 0); + assert(pSyncNode->pSyncRespMgr != NULL); + + // start in syncNodeStart + // start raft + // syncNodeBecomeFollower(pSyncNode); + + return pSyncNode; +} + +void syncNodeStart(SSyncNode* pSyncNode) { // start raft syncNodeBecomeFollower(pSyncNode); - return pSyncNode; + // for test + int32_t ret = 0; + // ret = syncNodeStartPingTimer(pSyncNode); + assert(ret == 0); } void syncNodeClose(SSyncNode* pSyncNode) { @@ -305,12 +477,18 @@ void syncNodeClose(SSyncNode* pSyncNode) { syncIndexMgrDestroy(pSyncNode->pNextIndex); syncIndexMgrDestroy(pSyncNode->pMatchIndex); logStoreDestory(pSyncNode->pLogStore); + raftCfgClose(pSyncNode->pRaftCfg); syncNodeStopPingTimer(pSyncNode); syncNodeStopElectTimer(pSyncNode); syncNodeStopHeartbeatTimer(pSyncNode); - taosMemoryFree(pSyncNode); + if (pSyncNode->pFsm != NULL) { + taosMemoryFree(pSyncNode->pFsm); + } + + // free memory in syncFreeNode + // taosMemoryFree(pSyncNode); } // ping -------------- @@ -328,7 +506,7 @@ int32_t syncNodePing(SSyncNode* pSyncNode, const SRaftId* destRaftId, SyncPing* int32_t syncNodePingSelf(SSyncNode* pSyncNode) { int32_t ret = 0; - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &pSyncNode->myRaftId); + SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &pSyncNode->myRaftId, pSyncNode->vgId); ret = syncNodePing(pSyncNode, &pMsg->destId, pMsg); assert(ret == 0); @@ -339,10 +517,9 @@ int32_t syncNodePingSelf(SSyncNode* pSyncNode) { int32_t syncNodePingPeers(SSyncNode* pSyncNode) { int32_t ret = 0; for (int i = 0; i < pSyncNode->peersNum; ++i) { - SRaftId destId; - syncUtilnodeInfo2raftId(&pSyncNode->peersNodeInfo[i], pSyncNode->vgId, &destId); - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &destId); - ret = syncNodePing(pSyncNode, &destId, pMsg); + SRaftId* destId = &(pSyncNode->peersId[i]); + SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, destId, pSyncNode->vgId); + ret = syncNodePing(pSyncNode, destId, pMsg); assert(ret == 0); syncPingDestroy(pMsg); } @@ -351,11 +528,10 @@ int32_t syncNodePingPeers(SSyncNode* pSyncNode) { int32_t syncNodePingAll(SSyncNode* pSyncNode) { int32_t ret = 0; - for (int i = 0; i < pSyncNode->syncCfg.replicaNum; ++i) { - SRaftId destId; - syncUtilnodeInfo2raftId(&pSyncNode->syncCfg.nodeInfo[i], pSyncNode->vgId, &destId); - SyncPing* pMsg = syncPingBuild3(&pSyncNode->myRaftId, &destId); - ret = syncNodePing(pSyncNode, &destId, pMsg); + for (int 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); } @@ -430,14 +606,30 @@ int32_t syncNodeStopHeartbeatTimer(SSyncNode* pSyncNode) { int32_t syncNodeSendMsgById(const SRaftId* destRaftId, SSyncNode* pSyncNode, SRpcMsg* pMsg) { SEpSet epSet; syncUtilraftId2EpSet(destRaftId, &epSet); - pSyncNode->FpSendMsg(pSyncNode->rpcClient, &epSet, pMsg); + if (pSyncNode->FpSendMsg != NULL) { + pMsg->noResp = 1; + // htonl + syncUtilMsgHtoN(pMsg->pCont); + + pSyncNode->FpSendMsg(pSyncNode->rpcClient, &epSet, pMsg); + } else { + sTrace("syncNodeSendMsgById pSyncNode->FpSendMsg is NULL"); + } return 0; } int32_t syncNodeSendMsgByInfo(const SNodeInfo* nodeInfo, SSyncNode* pSyncNode, SRpcMsg* pMsg) { SEpSet epSet; syncUtilnodeInfo2EpSet(nodeInfo, &epSet); - pSyncNode->FpSendMsg(pSyncNode->rpcClient, &epSet, pMsg); + if (pSyncNode->FpSendMsg != NULL) { + pMsg->noResp = 1; + // htonl + syncUtilMsgHtoN(pMsg->pCont); + + pSyncNode->FpSendMsg(pSyncNode->rpcClient, &epSet, pMsg); + } else { + sTrace("syncNodeSendMsgByInfo pSyncNode->FpSendMsg is NULL"); + } return 0; } @@ -448,7 +640,11 @@ cJSON* syncNode2Json(const SSyncNode* pSyncNode) { 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); @@ -501,9 +697,7 @@ cJSON* syncNode2Json(const SSyncNode* pSyncNode) { // tla+ server vars cJSON_AddNumberToObject(pRoot, "state", pSyncNode->state); cJSON_AddStringToObject(pRoot, "state_str", syncUtilState2String(pSyncNode->state)); - char tmpBuf[RAFT_STORE_BLOCK_SIZE]; - raftStoreSerialize(pSyncNode->pRaftStore, tmpBuf, sizeof(tmpBuf)); - cJSON_AddStringToObject(pRoot, "pRaftStore", tmpBuf); + cJSON_AddItemToObject(pRoot, "pRaftStore", raftStore2Json(pSyncNode->pRaftStore)); // tla+ candidate vars cJSON_AddItemToObject(pRoot, "pVotesGranted", voteGranted2Json(pSyncNode->pVotesGranted)); @@ -518,6 +712,11 @@ cJSON* syncNode2Json(const SSyncNode* pSyncNode) { 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); @@ -586,6 +785,19 @@ char* syncNode2Str(const SSyncNode* pSyncNode) { return serialized; } +char* syncNode2SimpleStr(const SSyncNode* pSyncNode) { + int len = 256; + char* s = (char*)taosMemoryMalloc(len); + snprintf(s, len, + "syncNode2SimpleStr vgId:%d currentTerm:%lu, commitIndex:%ld, state:%d %s, electTimerLogicClock:%lu, " + "electTimerLogicClockUser:%lu, " + "electTimerMS:%d", + pSyncNode->vgId, pSyncNode->pRaftStore->currentTerm, pSyncNode->commitIndex, pSyncNode->state, + syncUtilState2String(pSyncNode->state), pSyncNode->electTimerLogicClock, pSyncNode->electTimerLogicClockUser, + pSyncNode->electTimerMS); + return s; +} + SSyncNode* syncNodeAcquire(int64_t rid) { SSyncNode* pNode = taosAcquireRef(tsNodeRefId, rid); if (pNode == NULL) { @@ -672,24 +884,36 @@ void syncNodeCandidate2Leader(SSyncNode* pSyncNode) { assert(voteGrantedMajority(pSyncNode->pVotesGranted)); syncNodeBecomeLeader(pSyncNode); + syncNodeLog2("==state change syncNodeCandidate2Leader==", pSyncNode); + // Raft 3.6.2 Committing entries from previous terms + + // use this now syncNodeAppendNoop(pSyncNode); + + // do not use this // syncNodeEqNoop(pSyncNode); } void syncNodeFollower2Candidate(SSyncNode* pSyncNode) { assert(pSyncNode->state == TAOS_SYNC_STATE_FOLLOWER); pSyncNode->state = TAOS_SYNC_STATE_CANDIDATE; + + syncNodeLog2("==state change syncNodeFollower2Candidate==", pSyncNode); } void syncNodeLeader2Follower(SSyncNode* pSyncNode) { assert(pSyncNode->state == TAOS_SYNC_STATE_LEADER); syncNodeBecomeFollower(pSyncNode); + + syncNodeLog2("==state change syncNodeLeader2Follower==", pSyncNode); } void syncNodeCandidate2Follower(SSyncNode* pSyncNode) { assert(pSyncNode->state == TAOS_SYNC_STATE_CANDIDATE); syncNodeBecomeFollower(pSyncNode); + + syncNodeLog2("==state change syncNodeCandidate2Follower==", pSyncNode); } // raft vote -------------- @@ -707,7 +931,7 @@ void syncNodeVoteForTerm(SSyncNode* pSyncNode, SyncTerm term, SRaftId* pRaftId) void syncNodeVoteForSelf(SSyncNode* pSyncNode) { syncNodeVoteForTerm(pSyncNode, pSyncNode->pRaftStore->currentTerm, &(pSyncNode->myRaftId)); - SyncRequestVoteReply* pMsg = syncRequestVoteReplyBuild(); + SyncRequestVoteReply* pMsg = syncRequestVoteReplyBuild(pSyncNode->vgId); pMsg->srcId = pSyncNode->myRaftId; pMsg->destId = pSyncNode->myRaftId; pMsg->term = pSyncNode->pRaftStore->currentTerm; @@ -735,13 +959,13 @@ void syncNodePrint2(char* s, SSyncNode* pObj) { void syncNodeLog(SSyncNode* pObj) { char* serialized = syncNode2Str(pObj); - sTrace("syncNodeLog | len:%lu | %s", strlen(serialized), serialized); + sTraceLong("syncNodeLog | len:%lu | %s", strlen(serialized), serialized); taosMemoryFree(serialized); } void syncNodeLog2(char* s, SSyncNode* pObj) { char* serialized = syncNode2Str(pObj); - sTrace("syncNodeLog2 | len:%lu | %s | %s", strlen(serialized), s, serialized); + sTraceLong("syncNodeLog2 | len:%lu | %s | %s", strlen(serialized), s, serialized); taosMemoryFree(serialized); } @@ -751,11 +975,15 @@ static void syncNodeEqPingTimer(void* param, void* tmrId) { SSyncNode* pSyncNode = (SSyncNode*)param; if (atomic_load_64(&pSyncNode->pingTimerLogicClockUser) <= atomic_load_64(&pSyncNode->pingTimerLogicClock)) { SyncTimeout* pSyncMsg = syncTimeoutBuild2(SYNC_TIMEOUT_PING, atomic_load_64(&pSyncNode->pingTimerLogicClock), - pSyncNode->pingTimerMS, pSyncNode); + pSyncNode->pingTimerMS, pSyncNode->vgId, pSyncNode); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pSyncMsg, &rpcMsg); syncRpcMsgLog2((char*)"==syncNodeEqPingTimer==", &rpcMsg); - pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + if (pSyncNode->FpEqMsg != NULL) { + pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + } else { + sTrace("syncNodeEqPingTimer pSyncNode->FpEqMsg is NULL"); + } syncTimeoutDestroy(pSyncMsg); taosTmrReset(syncNodeEqPingTimer, pSyncNode->pingTimerMS, pSyncNode, gSyncEnv->pTimerManager, @@ -770,17 +998,21 @@ static void syncNodeEqElectTimer(void* param, void* tmrId) { SSyncNode* pSyncNode = (SSyncNode*)param; if (atomic_load_64(&pSyncNode->electTimerLogicClockUser) <= atomic_load_64(&pSyncNode->electTimerLogicClock)) { SyncTimeout* pSyncMsg = syncTimeoutBuild2(SYNC_TIMEOUT_ELECTION, atomic_load_64(&pSyncNode->electTimerLogicClock), - pSyncNode->electTimerMS, pSyncNode); + pSyncNode->electTimerMS, pSyncNode->vgId, pSyncNode); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pSyncMsg, &rpcMsg); syncRpcMsgLog2((char*)"==syncNodeEqElectTimer==", &rpcMsg); - pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + if (pSyncNode->FpEqMsg != NULL) { + pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + } else { + sTrace("syncNodeEqElectTimer pSyncNode->FpEqMsg is NULL"); + } syncTimeoutDestroy(pSyncMsg); // reset timer ms pSyncNode->electTimerMS = syncUtilElectRandomMS(pSyncNode->electBaseLine, 2 * pSyncNode->electBaseLine); - taosTmrReset(syncNodeEqPingTimer, pSyncNode->pingTimerMS, pSyncNode, gSyncEnv->pTimerManager, - &pSyncNode->pPingTimer); + taosTmrReset(syncNodeEqElectTimer, pSyncNode->electTimerMS, pSyncNode, gSyncEnv->pTimerManager, + &pSyncNode->pElectTimer); } else { sTrace("==syncNodeEqElectTimer== electTimerLogicClock:%" PRIu64 ", electTimerLogicClockUser:%" PRIu64 "", pSyncNode->electTimerLogicClock, pSyncNode->electTimerLogicClockUser); @@ -793,11 +1025,15 @@ static void syncNodeEqHeartbeatTimer(void* param, void* tmrId) { atomic_load_64(&pSyncNode->heartbeatTimerLogicClock)) { SyncTimeout* pSyncMsg = syncTimeoutBuild2(SYNC_TIMEOUT_HEARTBEAT, atomic_load_64(&pSyncNode->heartbeatTimerLogicClock), - pSyncNode->heartbeatTimerMS, pSyncNode); + pSyncNode->heartbeatTimerMS, pSyncNode->vgId, pSyncNode); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pSyncMsg, &rpcMsg); syncRpcMsgLog2((char*)"==syncNodeEqHeartbeatTimer==", &rpcMsg); - pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + if (pSyncNode->FpEqMsg != NULL) { + pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); + } else { + sTrace("syncNodeEqHeartbeatTimer pSyncNode->FpEqMsg is NULL"); + } syncTimeoutDestroy(pSyncMsg); taosTmrReset(syncNodeEqHeartbeatTimer, pSyncNode->heartbeatTimerMS, pSyncNode, gSyncEnv->pTimerManager, @@ -815,7 +1051,7 @@ static int32_t syncNodeEqNoop(SSyncNode* ths) { SyncIndex index = ths->pLogStore->getLastIndex(ths->pLogStore) + 1; SyncTerm term = ths->pRaftStore->currentTerm; - SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index); + SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index, ths->vgId); assert(pEntry != NULL); uint32_t entryLen; @@ -826,7 +1062,11 @@ static int32_t syncNodeEqNoop(SSyncNode* ths) { SRpcMsg rpcMsg; syncClientRequest2RpcMsg(pSyncMsg, &rpcMsg); - ths->FpEqMsg(ths->queue, &rpcMsg); + if (ths->FpEqMsg != NULL) { + ths->FpEqMsg(ths->queue, &rpcMsg); + } else { + sTrace("syncNodeEqNoop pSyncNode->FpEqMsg is NULL"); + } taosMemoryFree(serialized); syncClientRequestDestroy(pSyncMsg); @@ -839,7 +1079,7 @@ static int32_t syncNodeAppendNoop(SSyncNode* ths) { SyncIndex index = ths->pLogStore->getLastIndex(ths->pLogStore) + 1; SyncTerm term = ths->pRaftStore->currentTerm; - SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index); + SSyncRaftEntry* pEntry = syncEntryBuildNoop(term, index, ths->vgId); assert(pEntry != NULL); if (ths->state == TAOS_SYNC_STATE_LEADER) { @@ -851,18 +1091,34 @@ static int32_t syncNodeAppendNoop(SSyncNode* ths) { } // on message ---- -static int32_t syncNodeOnPingCb(SSyncNode* ths, SyncPing* pMsg) { +int32_t syncNodeOnPingCb(SSyncNode* ths, SyncPing* pMsg) { + // log state + char logBuf[1024]; + snprintf(logBuf, sizeof(logBuf), + "==syncNodeOnPingCb== vgId:%d, state: %d, %s, term:%lu electTimerLogicClock:%lu, " + "electTimerLogicClockUser:%lu, electTimerMS:%d", + ths->vgId, ths->state, syncUtilState2String(ths->state), ths->pRaftStore->currentTerm, + ths->electTimerLogicClock, ths->electTimerLogicClockUser, ths->electTimerMS); + int32_t ret = 0; - syncPingLog2("==syncNodeOnPingCb==", pMsg); - SyncPingReply* pMsgReply = syncPingReplyBuild3(&ths->myRaftId, &pMsg->srcId); + syncPingLog2(logBuf, pMsg); + SyncPingReply* pMsgReply = syncPingReplyBuild3(&ths->myRaftId, &pMsg->srcId, ths->vgId); SRpcMsg rpcMsg; syncPingReply2RpcMsg(pMsgReply, &rpcMsg); + + /* + // htonl + SMsgHead* pHead = rpcMsg.pCont; + pHead->contLen = htonl(pHead->contLen); + pHead->vgId = htonl(pHead->vgId); + */ + syncNodeSendMsgById(&pMsgReply->destId, ths, &rpcMsg); return ret; } -static int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg) { +int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg) { int32_t ret = 0; syncPingReplyLog2("==syncNodeOnPingReplyCb==", pMsg); return ret; @@ -878,7 +1134,7 @@ static int32_t syncNodeOnPingReplyCb(SSyncNode* ths, SyncPingReply* pMsg) { // /\ UNCHANGED <> // -static int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg) { +int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg) { int32_t ret = 0; syncClientRequestLog2("==syncNodeOnClientRequestCb==", pMsg); @@ -898,8 +1154,14 @@ static int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg syncEntry2OriginalRpc(pEntry, &rpcMsg); if (ths->pFsm != NULL) { - if (ths->pFsm->FpPreCommitCb != NULL && pEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, pEntry->index, pEntry->isWeak, 0, ths->state); + if (ths->pFsm->FpPreCommitCb != NULL && pEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pEntry->index; + cbMeta.isWeak = pEntry->isWeak; + cbMeta.code = 0; + cbMeta.state = ths->state; + cbMeta.seqNum = pEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); } } rpcFreeCont(rpcMsg.pCont); @@ -913,8 +1175,14 @@ static int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg syncEntry2OriginalRpc(pEntry, &rpcMsg); if (ths->pFsm != NULL) { - if (ths->pFsm->FpPreCommitCb != NULL && pEntry->entryType == SYNC_RAFT_ENTRY_DATA) { - ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, pEntry->index, pEntry->isWeak, 1, ths->state); + if (ths->pFsm->FpPreCommitCb != NULL && pEntry->originalRpcType != TDMT_VND_SYNC_NOOP) { + SFsmCbMeta cbMeta; + cbMeta.index = pEntry->index; + cbMeta.isWeak = pEntry->isWeak; + cbMeta.code = 1; + cbMeta.state = ths->state; + cbMeta.seqNum = pEntry->seqNum; + ths->pFsm->FpPreCommitCb(ths->pFsm, &rpcMsg, cbMeta); } } rpcFreeCont(rpcMsg.pCont); @@ -926,7 +1194,8 @@ static int32_t syncNodeOnClientRequestCb(SSyncNode* ths, SyncClientRequest* pMsg static void syncFreeNode(void* param) { SSyncNode* pNode = param; - syncNodePrint2((char*)"==syncFreeNode==", pNode); + // inner object already free + // syncNodePrint2((char*)"==syncFreeNode==", pNode); taosMemoryFree(pNode); -} \ No newline at end of file +} diff --git a/source/libs/sync/src/syncMessage.c b/source/libs/sync/src/syncMessage.c index 5dca165455..76c68569c3 100644 --- a/source/libs/sync/src/syncMessage.c +++ b/source/libs/sync/src/syncMessage.c @@ -22,50 +22,50 @@ cJSON* syncRpcMsg2Json(SRpcMsg* pRpcMsg) { cJSON* pRoot; // in compiler optimization, switch case = if else constants - if (pRpcMsg->msgType == SYNC_TIMEOUT) { + if (pRpcMsg->msgType == TDMT_VND_SYNC_TIMEOUT) { SyncTimeout* pSyncMsg = syncTimeoutDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncTimeout2Json(pSyncMsg); syncTimeoutDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_PING) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_PING) { SyncPing* pSyncMsg = syncPingDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncPing2Json(pSyncMsg); syncPingDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_PING_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_PING_REPLY) { SyncPingReply* pSyncMsg = syncPingReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncPingReply2Json(pSyncMsg); syncPingReplyDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_CLIENT_REQUEST) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_CLIENT_REQUEST) { SyncClientRequest* pSyncMsg = syncClientRequestDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncClientRequest2Json(pSyncMsg); syncClientRequestDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_CLIENT_REQUEST_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_CLIENT_REQUEST_REPLY) { pRoot = syncRpcUnknownMsg2Json(); - } else if (pRpcMsg->msgType == SYNC_REQUEST_VOTE) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_REQUEST_VOTE) { SyncRequestVote* pSyncMsg = syncRequestVoteDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncRequestVote2Json(pSyncMsg); syncRequestVoteDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_REQUEST_VOTE_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_REQUEST_VOTE_REPLY) { SyncRequestVoteReply* pSyncMsg = syncRequestVoteReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncRequestVoteReply2Json(pSyncMsg); syncRequestVoteReplyDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_APPEND_ENTRIES) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_APPEND_ENTRIES) { SyncAppendEntries* pSyncMsg = syncAppendEntriesDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncAppendEntries2Json(pSyncMsg); syncAppendEntriesDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_APPEND_ENTRIES_REPLY) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_APPEND_ENTRIES_REPLY) { SyncAppendEntriesReply* pSyncMsg = syncAppendEntriesReplyDeserialize2(pRpcMsg->pCont, pRpcMsg->contLen); pRoot = syncAppendEntriesReply2Json(pSyncMsg); syncAppendEntriesReplyDestroy(pSyncMsg); - } else if (pRpcMsg->msgType == SYNC_RESPONSE) { + } else if (pRpcMsg->msgType == TDMT_VND_SYNC_COMMON_RESPONSE) { pRoot = cJSON_CreateObject(); char* s; s = syncUtilprintBin((char*)(pRpcMsg->pCont), pRpcMsg->contLen); @@ -98,7 +98,7 @@ cJSON* syncRpcMsg2Json(SRpcMsg* pRpcMsg) { cJSON* syncRpcUnknownMsg2Json() { cJSON* pRoot = cJSON_CreateObject(); - cJSON_AddNumberToObject(pRoot, "msgType", SYNC_UNKNOWN); + cJSON_AddNumberToObject(pRoot, "msgType", TDMT_VND_SYNC_UNKNOWN); cJSON_AddStringToObject(pRoot, "data", "unknown message"); cJSON* pJson = cJSON_CreateObject(); @@ -146,12 +146,14 @@ SyncTimeout* syncTimeoutBuild() { SyncTimeout* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_TIMEOUT; + pMsg->msgType = TDMT_VND_SYNC_TIMEOUT; return pMsg; } -SyncTimeout* syncTimeoutBuild2(ESyncTimeoutType timeoutType, uint64_t logicClock, int32_t timerMS, void* data) { +SyncTimeout* syncTimeoutBuild2(ESyncTimeoutType timeoutType, uint64_t logicClock, int32_t timerMS, int32_t vgId, + void* data) { SyncTimeout* pMsg = syncTimeoutBuild(); + pMsg->vgId = vgId; pMsg->timeoutType = timeoutType; pMsg->logicClock = logicClock; pMsg->timerMS = timerMS; @@ -217,6 +219,7 @@ cJSON* syncTimeout2Json(const SyncTimeout* pMsg) { if (pMsg != NULL) { cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); + cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); cJSON_AddNumberToObject(pRoot, "timeoutType", pMsg->timeoutType); snprintf(u64buf, sizeof(u64buf), "%lu", pMsg->logicClock); @@ -271,22 +274,23 @@ SyncPing* syncPingBuild(uint32_t dataLen) { SyncPing* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_PING; + pMsg->msgType = TDMT_VND_SYNC_PING; pMsg->dataLen = dataLen; return pMsg; } -SyncPing* syncPingBuild2(const SRaftId* srcId, const SRaftId* destId, const char* str) { +SyncPing* syncPingBuild2(const SRaftId* srcId, const SRaftId* destId, int32_t vgId, const char* str) { uint32_t dataLen = strlen(str) + 1; SyncPing* pMsg = syncPingBuild(dataLen); + pMsg->vgId = vgId; pMsg->srcId = *srcId; pMsg->destId = *destId; snprintf(pMsg->data, pMsg->dataLen, "%s", str); return pMsg; } -SyncPing* syncPingBuild3(const SRaftId* srcId, const SRaftId* destId) { - SyncPing* pMsg = syncPingBuild2(srcId, destId, "ping"); +SyncPing* syncPingBuild3(const SRaftId* srcId, const SRaftId* destId, int32_t vgId) { + SyncPing* pMsg = syncPingBuild2(srcId, destId, vgId, "ping"); return pMsg; } @@ -326,6 +330,98 @@ SyncPing* syncPingDeserialize2(const char* buf, uint32_t len) { return pMsg; } +int32_t syncPingSerialize3(const SyncPing* pMsg, char* buf, int32_t bufLen) { + SCoder encoder = {0}; + tCoderInit(&encoder, TD_LITTLE_ENDIAN, buf, bufLen, TD_ENCODER); + if (tStartEncode(&encoder) < 0) { + return -1; + } + + if (tEncodeU32(&encoder, pMsg->bytes) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->vgId) < 0) { + return -1; + } + if (tEncodeU32(&encoder, pMsg->msgType) < 0) { + return -1; + } + if (tEncodeU64(&encoder, pMsg->srcId.addr) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->srcId.vgId) < 0) { + return -1; + } + if (tEncodeU64(&encoder, pMsg->destId.addr) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->destId.vgId) < 0) { + return -1; + } + if (tEncodeU32(&encoder, pMsg->dataLen) < 0) { + return -1; + } + if (tEncodeBinary(&encoder, pMsg->data, pMsg->dataLen)) { + return -1; + } + + tEndEncode(&encoder); + int32_t tlen = encoder.pos; + tCoderClear(&encoder); + return tlen; +} + +SyncPing* syncPingDeserialize3(void* buf, int32_t bufLen) { + SCoder decoder = {0}; + tCoderInit(&decoder, TD_LITTLE_ENDIAN, buf, bufLen, TD_DECODER); + if (tStartDecode(&decoder) < 0) { + return NULL; + } + + SyncPing* pMsg = NULL; + uint32_t bytes; + if (tDecodeU32(&decoder, &bytes) < 0) { + return NULL; + } + + pMsg = taosMemoryMalloc(bytes); + assert(pMsg != NULL); + pMsg->bytes = bytes; + + if (tDecodeI32(&decoder, &pMsg->vgId) < 0) { + return NULL; + } + if (tDecodeU32(&decoder, &pMsg->msgType) < 0) { + return NULL; + } + if (tDecodeU64(&decoder, &pMsg->srcId.addr) < 0) { + return NULL; + } + if (tDecodeI32(&decoder, &pMsg->srcId.vgId) < 0) { + return NULL; + } + if (tDecodeU64(&decoder, &pMsg->destId.addr) < 0) { + return NULL; + } + if (tDecodeI32(&decoder, &pMsg->destId.vgId) < 0) { + return NULL; + } + if (tDecodeU32(&decoder, &pMsg->dataLen) < 0) { + return NULL; + } + uint64_t len; + char* data = NULL; + if (tDecodeBinary(&decoder, (const void**)(&data), &len) < 0) { + return NULL; + } + assert(len = pMsg->dataLen); + memcpy(pMsg->data, data, len); + + tEndDecode(&decoder); + tCoderClear(&decoder); + return pMsg; +} + void syncPing2RpcMsg(const SyncPing* pMsg, SRpcMsg* pRpcMsg) { memset(pRpcMsg, 0, sizeof(*pRpcMsg)); pRpcMsg->msgType = pMsg->msgType; @@ -349,6 +445,7 @@ cJSON* syncPing2Json(const SyncPing* pMsg) { 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(); @@ -436,22 +533,23 @@ SyncPingReply* syncPingReplyBuild(uint32_t dataLen) { SyncPingReply* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_PING_REPLY; + pMsg->msgType = TDMT_VND_SYNC_PING_REPLY; pMsg->dataLen = dataLen; return pMsg; } -SyncPingReply* syncPingReplyBuild2(const SRaftId* srcId, const SRaftId* destId, const char* str) { +SyncPingReply* syncPingReplyBuild2(const SRaftId* srcId, const SRaftId* destId, int32_t vgId, const char* str) { uint32_t dataLen = strlen(str) + 1; SyncPingReply* pMsg = syncPingReplyBuild(dataLen); + pMsg->vgId = vgId; pMsg->srcId = *srcId; pMsg->destId = *destId; snprintf(pMsg->data, pMsg->dataLen, "%s", str); return pMsg; } -SyncPingReply* syncPingReplyBuild3(const SRaftId* srcId, const SRaftId* destId) { - SyncPingReply* pMsg = syncPingReplyBuild2(srcId, destId, "pang"); +SyncPingReply* syncPingReplyBuild3(const SRaftId* srcId, const SRaftId* destId, int32_t vgId) { + SyncPingReply* pMsg = syncPingReplyBuild2(srcId, destId, vgId, "pang"); return pMsg; } @@ -491,6 +589,98 @@ SyncPingReply* syncPingReplyDeserialize2(const char* buf, uint32_t len) { return pMsg; } +int32_t syncPingReplySerialize3(const SyncPingReply* pMsg, char* buf, int32_t bufLen) { + SCoder encoder = {0}; + tCoderInit(&encoder, TD_LITTLE_ENDIAN, buf, bufLen, TD_ENCODER); + if (tStartEncode(&encoder) < 0) { + return -1; + } + + if (tEncodeU32(&encoder, pMsg->bytes) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->vgId) < 0) { + return -1; + } + if (tEncodeU32(&encoder, pMsg->msgType) < 0) { + return -1; + } + if (tEncodeU64(&encoder, pMsg->srcId.addr) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->srcId.vgId) < 0) { + return -1; + } + if (tEncodeU64(&encoder, pMsg->destId.addr) < 0) { + return -1; + } + if (tEncodeI32(&encoder, pMsg->destId.vgId) < 0) { + return -1; + } + if (tEncodeU32(&encoder, pMsg->dataLen) < 0) { + return -1; + } + if (tEncodeBinary(&encoder, pMsg->data, pMsg->dataLen)) { + return -1; + } + + tEndEncode(&encoder); + int32_t tlen = encoder.pos; + tCoderClear(&encoder); + return tlen; +} + +SyncPingReply* syncPingReplyDeserialize3(void* buf, int32_t bufLen) { + SCoder decoder = {0}; + tCoderInit(&decoder, TD_LITTLE_ENDIAN, buf, bufLen, TD_DECODER); + if (tStartDecode(&decoder) < 0) { + return NULL; + } + + SyncPingReply* pMsg = NULL; + uint32_t bytes; + if (tDecodeU32(&decoder, &bytes) < 0) { + return NULL; + } + + pMsg = taosMemoryMalloc(bytes); + assert(pMsg != NULL); + pMsg->bytes = bytes; + + if (tDecodeI32(&decoder, &pMsg->vgId) < 0) { + return NULL; + } + if (tDecodeU32(&decoder, &pMsg->msgType) < 0) { + return NULL; + } + if (tDecodeU64(&decoder, &pMsg->srcId.addr) < 0) { + return NULL; + } + if (tDecodeI32(&decoder, &pMsg->srcId.vgId) < 0) { + return NULL; + } + if (tDecodeU64(&decoder, &pMsg->destId.addr) < 0) { + return NULL; + } + if (tDecodeI32(&decoder, &pMsg->destId.vgId) < 0) { + return NULL; + } + if (tDecodeU32(&decoder, &pMsg->dataLen) < 0) { + return NULL; + } + uint64_t len; + char* data = NULL; + if (tDecodeBinary(&decoder, (const void**)(&data), &len) < 0) { + return NULL; + } + assert(len = pMsg->dataLen); + memcpy(pMsg->data, data, len); + + tEndDecode(&decoder); + tCoderClear(&decoder); + return pMsg; +} + void syncPingReply2RpcMsg(const SyncPingReply* pMsg, SRpcMsg* pRpcMsg) { memset(pRpcMsg, 0, sizeof(*pRpcMsg)); pRpcMsg->msgType = pMsg->msgType; @@ -514,6 +704,7 @@ cJSON* syncPingReply2Json(const SyncPingReply* pMsg) { 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(); @@ -601,7 +792,7 @@ SyncClientRequest* syncClientRequestBuild(uint32_t dataLen) { SyncClientRequest* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_CLIENT_REQUEST; + pMsg->msgType = TDMT_VND_SYNC_CLIENT_REQUEST; pMsg->seqNum = 0; pMsg->isWeak = false; pMsg->dataLen = dataLen; @@ -609,8 +800,9 @@ SyncClientRequest* syncClientRequestBuild(uint32_t dataLen) { } // step 1. original SRpcMsg => SyncClientRequest, add seqNum, isWeak -SyncClientRequest* syncClientRequestBuild2(const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, bool isWeak) { +SyncClientRequest* syncClientRequestBuild2(const SRpcMsg* pOriginalRpcMsg, uint64_t seqNum, bool isWeak, int32_t vgId) { SyncClientRequest* pMsg = syncClientRequestBuild(pOriginalRpcMsg->contLen); + pMsg->vgId = vgId; pMsg->originalRpcType = pOriginalRpcMsg->msgType; pMsg->seqNum = seqNum; pMsg->isWeak = isWeak; @@ -678,6 +870,7 @@ cJSON* syncClientRequest2Json(const SyncClientRequest* pMsg) { if (pMsg != NULL) { cJSON_AddNumberToObject(pRoot, "bytes", pMsg->bytes); + cJSON_AddNumberToObject(pRoot, "vgId", pMsg->vgId); cJSON_AddNumberToObject(pRoot, "msgType", pMsg->msgType); cJSON_AddNumberToObject(pRoot, "originalRpcType", pMsg->originalRpcType); snprintf(u64buf, sizeof(u64buf), "%lu", pMsg->seqNum); @@ -734,12 +927,13 @@ void syncClientRequestLog2(char* s, const SyncClientRequest* pMsg) { } // ---- message process SyncRequestVote---- -SyncRequestVote* syncRequestVoteBuild() { +SyncRequestVote* syncRequestVoteBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncRequestVote); SyncRequestVote* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_REQUEST_VOTE; + pMsg->vgId = vgId; + pMsg->msgType = TDMT_VND_SYNC_REQUEST_VOTE; return pMsg; } @@ -801,6 +995,7 @@ cJSON* syncRequestVote2Json(const SyncRequestVote* pMsg) { 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(); @@ -880,12 +1075,13 @@ void syncRequestVoteLog2(char* s, const SyncRequestVote* pMsg) { } // ---- message process SyncRequestVoteReply---- -SyncRequestVoteReply* syncRequestVoteReplyBuild() { +SyncRequestVoteReply* syncRequestVoteReplyBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncRequestVoteReply); SyncRequestVoteReply* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_REQUEST_VOTE_REPLY; + pMsg->vgId = vgId; + pMsg->msgType = TDMT_VND_SYNC_REQUEST_VOTE_REPLY; return pMsg; } @@ -947,6 +1143,7 @@ cJSON* syncRequestVoteReply2Json(const SyncRequestVoteReply* pMsg) { 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(); @@ -1023,12 +1220,13 @@ void syncRequestVoteReplyLog2(char* s, const SyncRequestVoteReply* pMsg) { } // ---- message process SyncAppendEntries---- -SyncAppendEntries* syncAppendEntriesBuild(uint32_t dataLen) { +SyncAppendEntries* syncAppendEntriesBuild(uint32_t dataLen, int32_t vgId) { uint32_t bytes = sizeof(SyncAppendEntries) + dataLen; SyncAppendEntries* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_APPEND_ENTRIES; + pMsg->vgId = vgId; + pMsg->msgType = TDMT_VND_SYNC_APPEND_ENTRIES; pMsg->dataLen = dataLen; return pMsg; } @@ -1092,6 +1290,7 @@ cJSON* syncAppendEntries2Json(const SyncAppendEntries* pMsg) { 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(); @@ -1186,12 +1385,13 @@ void syncAppendEntriesLog2(char* s, const SyncAppendEntries* pMsg) { } // ---- message process SyncAppendEntriesReply---- -SyncAppendEntriesReply* syncAppendEntriesReplyBuild() { +SyncAppendEntriesReply* syncAppendEntriesReplyBuild(int32_t vgId) { uint32_t bytes = sizeof(SyncAppendEntriesReply); SyncAppendEntriesReply* pMsg = taosMemoryMalloc(bytes); memset(pMsg, 0, bytes); pMsg->bytes = bytes; - pMsg->msgType = SYNC_APPEND_ENTRIES_REPLY; + pMsg->vgId = vgId; + pMsg->msgType = TDMT_VND_SYNC_APPEND_ENTRIES_REPLY; return pMsg; } @@ -1253,6 +1453,7 @@ cJSON* syncAppendEntriesReply2Json(const SyncAppendEntriesReply* pMsg) { 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(); diff --git a/source/libs/sync/src/syncRaftCfg.c b/source/libs/sync/src/syncRaftCfg.c new file mode 100644 index 0000000000..dc540424ec --- /dev/null +++ b/source/libs/sync/src/syncRaftCfg.c @@ -0,0 +1,211 @@ +/* + * 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 . + */ + +#include "syncRaftCfg.h" +#include "cJSON.h" +#include "syncEnv.h" +#include "syncUtil.h" + +// file must already exist! +SRaftCfg *raftCfgOpen(const char *path) { + SRaftCfg *pCfg = taosMemoryMalloc(sizeof(SRaftCfg)); + snprintf(pCfg->path, sizeof(pCfg->path), "%s", path); + + pCfg->pFile = taosOpenFile(pCfg->path, TD_FILE_READ | TD_FILE_WRITE); + assert(pCfg->pFile != NULL); + + taosLSeekFile(pCfg->pFile, 0, SEEK_SET); + + char buf[1024]; + int len = taosReadFile(pCfg->pFile, buf, sizeof(buf)); + assert(len > 0); + + int32_t ret = syncCfgFromStr(buf, &(pCfg->cfg)); + assert(ret == 0); + + return pCfg; +} + +int32_t raftCfgClose(SRaftCfg *pRaftCfg) { + int64_t ret = taosCloseFile(&(pRaftCfg->pFile)); + assert(ret == 0); + taosMemoryFree(pRaftCfg); + return 0; +} + +int32_t raftCfgPersist(SRaftCfg *pRaftCfg) { + assert(pRaftCfg != NULL); + + char *s = syncCfg2Str(&(pRaftCfg->cfg)); + taosLSeekFile(pRaftCfg->pFile, 0, SEEK_SET); + int64_t ret = taosWriteFile(pRaftCfg->pFile, s, strlen(s) + 1); + assert(ret == strlen(s) + 1); + taosMemoryFree(s); + + taosFsyncFile(pRaftCfg->pFile); + return 0; +} + +cJSON *syncCfg2Json(SSyncCfg *pSyncCfg) { + char u64buf[128]; + cJSON *pRoot = cJSON_CreateObject(); + + if (pSyncCfg != NULL) { + cJSON_AddNumberToObject(pRoot, "replicaNum", pSyncCfg->replicaNum); + cJSON_AddNumberToObject(pRoot, "myIndex", pSyncCfg->myIndex); + + cJSON *pNodeInfoArr = cJSON_CreateArray(); + cJSON_AddItemToObject(pRoot, "nodeInfo", pNodeInfoArr); + for (int i = 0; i < pSyncCfg->replicaNum; ++i) { + cJSON *pNodeInfo = cJSON_CreateObject(); + cJSON_AddNumberToObject(pNodeInfo, "nodePort", ((pSyncCfg->nodeInfo)[i]).nodePort); + cJSON_AddStringToObject(pNodeInfo, "nodeFqdn", ((pSyncCfg->nodeInfo)[i]).nodeFqdn); + cJSON_AddItemToArray(pNodeInfoArr, pNodeInfo); + } + } + + cJSON *pJson = cJSON_CreateObject(); + cJSON_AddItemToObject(pJson, "SSyncCfg", pRoot); + return pJson; +} + +char *syncCfg2Str(SSyncCfg *pSyncCfg) { + cJSON *pJson = syncCfg2Json(pSyncCfg); + char * serialized = cJSON_Print(pJson); + cJSON_Delete(pJson); + return serialized; +} + +int32_t syncCfgFromJson(const cJSON *pRoot, SSyncCfg *pSyncCfg) { + memset(pSyncCfg, 0, sizeof(SSyncCfg)); + cJSON *pJson = cJSON_GetObjectItem(pRoot, "SSyncCfg"); + + cJSON *pReplicaNum = cJSON_GetObjectItem(pJson, "replicaNum"); + assert(cJSON_IsNumber(pReplicaNum)); + pSyncCfg->replicaNum = cJSON_GetNumberValue(pReplicaNum); + + cJSON *pMyIndex = cJSON_GetObjectItem(pJson, "myIndex"); + assert(cJSON_IsNumber(pMyIndex)); + pSyncCfg->myIndex = cJSON_GetNumberValue(pMyIndex); + + cJSON *pNodeInfoArr = cJSON_GetObjectItem(pJson, "nodeInfo"); + int arraySize = cJSON_GetArraySize(pNodeInfoArr); + assert(arraySize == pSyncCfg->replicaNum); + + for (int i = 0; i < arraySize; ++i) { + cJSON *pNodeInfo = cJSON_GetArrayItem(pNodeInfoArr, i); + assert(pNodeInfo != NULL); + + cJSON *pNodePort = cJSON_GetObjectItem(pNodeInfo, "nodePort"); + assert(cJSON_IsNumber(pNodePort)); + ((pSyncCfg->nodeInfo)[i]).nodePort = cJSON_GetNumberValue(pNodePort); + + cJSON *pNodeFqdn = cJSON_GetObjectItem(pNodeInfo, "nodeFqdn"); + assert(cJSON_IsString(pNodeFqdn)); + snprintf(((pSyncCfg->nodeInfo)[i]).nodeFqdn, sizeof(((pSyncCfg->nodeInfo)[i]).nodeFqdn), "%s", + pNodeFqdn->valuestring); + } + + return 0; +} + +int32_t syncCfgFromStr(const char *s, SSyncCfg *pSyncCfg) { + cJSON *pRoot = cJSON_Parse(s); + assert(pRoot != NULL); + + int32_t ret = syncCfgFromJson(pRoot, pSyncCfg); + assert(ret == 0); + + cJSON_Delete(pRoot); + return 0; +} + +cJSON *raftCfg2Json(SRaftCfg *pRaftCfg) { + cJSON *pJson = syncCfg2Json(&(pRaftCfg->cfg)); + return pJson; +} + +char *raftCfg2Str(SRaftCfg *pRaftCfg) { + char *s = syncCfg2Str(&(pRaftCfg->cfg)); + return s; +} + +int32_t syncCfgCreateFile(SSyncCfg *pCfg, const char *path) { + assert(pCfg != NULL); + + TdFilePtr pFile = taosOpenFile(path, TD_FILE_CREATE | TD_FILE_WRITE); + assert(pFile != NULL); + + char * s = syncCfg2Str(pCfg); + int64_t ret = taosWriteFile(pFile, s, strlen(s) + 1); + assert(ret == strlen(s) + 1); + + taosMemoryFree(s); + taosCloseFile(&pFile); + return 0; +} + +// for debug ---------------------- +void syncCfgPrint(SSyncCfg *pCfg) { + char *serialized = syncCfg2Str(pCfg); + printf("syncCfgPrint | len:%lu | %s \n", strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncCfgPrint2(char *s, SSyncCfg *pCfg) { + char *serialized = syncCfg2Str(pCfg); + printf("syncCfgPrint2 | len:%lu | %s | %s \n", strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void syncCfgLog(SSyncCfg *pCfg) { + char *serialized = syncCfg2Str(pCfg); + sTrace("syncCfgLog | len:%lu | %s", strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void syncCfgLog2(char *s, SSyncCfg *pCfg) { + char *serialized = syncCfg2Str(pCfg); + sTrace("syncCfgLog2 | len:%lu | %s | %s", strlen(serialized), s, serialized); + taosMemoryFree(serialized); +} + +void raftCfgPrint(SRaftCfg *pCfg) { + char *serialized = raftCfg2Str(pCfg); + printf("raftCfgPrint | len:%lu | %s \n", strlen(serialized), serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftCfgPrint2(char *s, SRaftCfg *pCfg) { + char *serialized = raftCfg2Str(pCfg); + printf("raftCfgPrint2 | len:%lu | %s | %s \n", strlen(serialized), s, serialized); + fflush(NULL); + taosMemoryFree(serialized); +} + +void raftCfgLog(SRaftCfg *pCfg) { + char *serialized = raftCfg2Str(pCfg); + sTrace("raftCfgLog | len:%lu | %s", strlen(serialized), serialized); + taosMemoryFree(serialized); +} + +void raftCfgLog2(char *s, SRaftCfg *pCfg) { + char *serialized = raftCfg2Str(pCfg); + sTrace("raftCfgLog2 | len:%lu | %s | %s", strlen(serialized), s, serialized); + taosMemoryFree(serialized); +} diff --git a/source/libs/sync/src/syncRaftEntry.c b/source/libs/sync/src/syncRaftEntry.c index 3de39a6aa8..21ee35eaf9 100644 --- a/source/libs/sync/src/syncRaftEntry.c +++ b/source/libs/sync/src/syncRaftEntry.c @@ -28,13 +28,13 @@ SSyncRaftEntry* syncEntryBuild(uint32_t dataLen) { // step 4. SyncClientRequest => SSyncRaftEntry, add term, index SSyncRaftEntry* syncEntryBuild2(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index) { - SSyncRaftEntry* pEntry = syncEntryBuild3(pMsg, term, index, SYNC_RAFT_ENTRY_DATA); + SSyncRaftEntry* pEntry = syncEntryBuild3(pMsg, term, index); assert(pEntry != NULL); return pEntry; } -SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index, EntryType entryType) { +SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncIndex index) { SSyncRaftEntry* pEntry = syncEntryBuild(pMsg->dataLen); assert(pEntry != NULL); @@ -44,19 +44,37 @@ SSyncRaftEntry* syncEntryBuild3(SyncClientRequest* pMsg, SyncTerm term, SyncInde pEntry->isWeak = pMsg->isWeak; pEntry->term = term; pEntry->index = index; - pEntry->entryType = entryType; pEntry->dataLen = pMsg->dataLen; memcpy(pEntry->data, pMsg->data, pMsg->dataLen); return pEntry; } -SSyncRaftEntry* syncEntryBuildNoop(SyncTerm term, SyncIndex index) { - SSyncRaftEntry* pEntry = syncEntryBuild(0); +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_VND_SYNC_NOOP; + memcpy(rpcMsg.pCont, &head, sizeof(head)); + + SSyncRaftEntry* pEntry = syncEntryBuild(rpcMsg.contLen); assert(pEntry != NULL); + + pEntry->msgType = TDMT_VND_SYNC_CLIENT_REQUEST; + pEntry->originalRpcType = TDMT_VND_SYNC_NOOP; + pEntry->seqNum = 0; + pEntry->isWeak = 0; pEntry->term = term; pEntry->index = index; - pEntry->entryType = SYNC_RAFT_ENTRY_NOOP; + + assert(pEntry->dataLen == rpcMsg.contLen); + memcpy(pEntry->data, rpcMsg.pCont, rpcMsg.contLen); + rpcFreeCont(rpcMsg.pCont); return pEntry; } @@ -103,7 +121,6 @@ cJSON* syncEntry2Json(const SSyncRaftEntry* pEntry) { cJSON_AddStringToObject(pRoot, "term", u64buf); snprintf(u64buf, sizeof(u64buf), "%lu", pEntry->index); cJSON_AddStringToObject(pRoot, "index", u64buf); - cJSON_AddNumberToObject(pRoot, "entryType", pEntry->entryType); cJSON_AddNumberToObject(pRoot, "dataLen", pEntry->dataLen); char* s; diff --git a/source/libs/sync/src/syncRaftLog.c b/source/libs/sync/src/syncRaftLog.c index 63c6265d10..dc903ad7b2 100644 --- a/source/libs/sync/src/syncRaftLog.c +++ b/source/libs/sync/src/syncRaftLog.c @@ -50,38 +50,46 @@ int32_t logStoreAppendEntry(SSyncLogStore* pLogStore, SSyncRaftEntry* pEntry) { SyncIndex lastIndex = logStoreLastIndex(pLogStore); assert(pEntry->index == lastIndex + 1); - uint32_t len; - char* serialized = syncEntrySerialize(pEntry, &len); - assert(serialized != NULL); - int code = 0; - /* - code = walWrite(pWal, pEntry->index, pEntry->entryType, serialized, len); - assert(code == 0); - */ - assert(walWrite(pWal, pEntry->index, pEntry->entryType, serialized, len) == 0); + int code = 0; + SSyncLogMeta syncMeta; + syncMeta.isWeek = pEntry->isWeak; + syncMeta.seqNum = pEntry->seqNum; + syncMeta.term = pEntry->term; + code = walWriteWithSyncInfo(pWal, pEntry->index, pEntry->originalRpcType, syncMeta, pEntry->data, pEntry->dataLen); + assert(code == 0); walFsync(pWal, true); - taosMemoryFree(serialized); return code; } SSyncRaftEntry* logStoreGetEntry(SSyncLogStore* pLogStore, SyncIndex index) { SSyncLogStoreData* pData = pLogStore->data; SWal* pWal = pData->pWal; - SSyncRaftEntry* pEntry = NULL; if (index >= SYNC_INDEX_BEGIN && index <= logStoreLastIndex(pLogStore)) { SWalReadHandle* pWalHandle = walOpenReadHandle(pWal); assert(walReadWithHandle(pWalHandle, index) == 0); - pEntry = syncEntryDeserialize(pWalHandle->pHead->head.body, pWalHandle->pHead->head.len); + + SSyncRaftEntry* pEntry = syncEntryBuild(pWalHandle->pHead->head.len); assert(pEntry != NULL); + pEntry->msgType = TDMT_VND_SYNC_CLIENT_REQUEST; + pEntry->originalRpcType = pWalHandle->pHead->head.msgType; + pEntry->seqNum = pWalHandle->pHead->head.syncMeta.seqNum; + pEntry->isWeak = pWalHandle->pHead->head.syncMeta.isWeek; + pEntry->term = pWalHandle->pHead->head.syncMeta.term; + pEntry->index = index; + assert(pEntry->dataLen == pWalHandle->pHead->head.len); + memcpy(pEntry->data, pWalHandle->pHead->head.body, pWalHandle->pHead->head.len); + // need to hold, do not new every time!! walCloseReadHandle(pWalHandle); - } + return pEntry; - return pEntry; + } else { + return NULL; + } } int32_t logStoreTruncate(SSyncLogStore* pLogStore, SyncIndex fromIndex) { @@ -207,20 +215,20 @@ void logStorePrint(SSyncLogStore* pLogStore) { void logStorePrint2(char* s, SSyncLogStore* pLogStore) { char* serialized = logStore2Str(pLogStore); - printf("logStorePrint | len:%lu | %s | %s \n", strlen(serialized), s, serialized); + printf("logStorePrint2 | len:%lu | %s | %s \n", strlen(serialized), s, serialized); fflush(NULL); taosMemoryFree(serialized); } void logStoreLog(SSyncLogStore* pLogStore) { char* serialized = logStore2Str(pLogStore); - sTrace("logStorePrint | len:%lu | %s", strlen(serialized), serialized); + sTraceLong("logStoreLog | len:%lu | %s", strlen(serialized), serialized); taosMemoryFree(serialized); } void logStoreLog2(char* s, SSyncLogStore* pLogStore) { char* serialized = logStore2Str(pLogStore); - sTrace("logStorePrint | len:%lu | %s | %s", strlen(serialized), s, serialized); + sTraceLong("logStoreLog2 | len:%lu | %s | %s", strlen(serialized), s, serialized); taosMemoryFree(serialized); } diff --git a/source/libs/sync/src/syncReplication.c b/source/libs/sync/src/syncReplication.c index 3ced7a18bc..2fdb8a0e17 100644 --- a/source/libs/sync/src/syncReplication.c +++ b/source/libs/sync/src/syncReplication.c @@ -77,7 +77,7 @@ int32_t syncNodeAppendEntriesPeers(SSyncNode* pSyncNode) { SyncAppendEntries* pMsg = NULL; SSyncRaftEntry* pEntry = logStoreGetEntry(pSyncNode->pLogStore, nextIndex); if (pEntry != NULL) { - pMsg = syncAppendEntriesBuild(pEntry->bytes); + pMsg = syncAppendEntriesBuild(pEntry->bytes, pSyncNode->vgId); assert(pMsg != NULL); // add pEntry into msg @@ -91,7 +91,7 @@ int32_t syncNodeAppendEntriesPeers(SSyncNode* pSyncNode) { } else { // maybe overflow, send empty record - pMsg = syncAppendEntriesBuild(0); + pMsg = syncAppendEntriesBuild(0, pSyncNode->vgId); assert(pMsg != NULL); } diff --git a/source/libs/sync/src/syncRequestVote.c b/source/libs/sync/src/syncRequestVote.c index e23748a81e..619a1546a9 100644 --- a/source/libs/sync/src/syncRequestVote.c +++ b/source/libs/sync/src/syncRequestVote.c @@ -64,7 +64,7 @@ int32_t syncNodeOnRequestVoteCb(SSyncNode* ths, SyncRequestVote* pMsg) { raftStoreVote(ths->pRaftStore, &(pMsg->srcId)); } - SyncRequestVoteReply* pReply = syncRequestVoteReplyBuild(); + SyncRequestVoteReply* pReply = syncRequestVoteReplyBuild(ths->vgId); pReply->srcId = ths->myRaftId; pReply->destId = pMsg->srcId; pReply->term = ths->pRaftStore->currentTerm; diff --git a/source/libs/sync/src/syncRequestVoteReply.c b/source/libs/sync/src/syncRequestVoteReply.c index 1531f701ff..a6348dff50 100644 --- a/source/libs/sync/src/syncRequestVoteReply.c +++ b/source/libs/sync/src/syncRequestVoteReply.c @@ -49,11 +49,20 @@ int32_t syncNodeOnRequestVoteReplyCb(SSyncNode* ths, SyncRequestVoteReply* pMsg) return ret; } - assert(!(pMsg->term > ths->pRaftStore->currentTerm)); - // no need this code, because if I receive reply.term, then I must have sent for that term. - // if (pMsg->term > ths->pRaftStore->currentTerm) { - // syncNodeUpdateTerm(ths, pMsg->term); - // } + // assert(!(pMsg->term > ths->pRaftStore->currentTerm)); + // no need this code, because if I receive reply.term, then I must have sent for that term. + // if (pMsg->term > ths->pRaftStore->currentTerm) { + // syncNodeUpdateTerm(ths, pMsg->term); + // } + + if (pMsg->term > ths->pRaftStore->currentTerm) { + char logBuf[128]; + snprintf(logBuf, sizeof(logBuf), "syncNodeOnRequestVoteReplyCb error term, receive:%lu current:%lu", pMsg->term, + ths->pRaftStore->currentTerm); + syncNodePrint2(logBuf, ths); + sError("%s", logBuf); + return ret; + } assert(pMsg->term == ths->pRaftStore->currentTerm); diff --git a/source/libs/sync/src/syncRespMgr.c b/source/libs/sync/src/syncRespMgr.c new file mode 100644 index 0000000000..47c73745fd --- /dev/null +++ b/source/libs/sync/src/syncRespMgr.c @@ -0,0 +1,93 @@ +/* + * 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 . + */ + +#include "syncRespMgr.h" + +SSyncRespMgr *syncRespMgrCreate(void *data, int64_t ttl) { + SSyncRespMgr *pObj = (SSyncRespMgr *)taosMemoryMalloc(sizeof(SSyncRespMgr)); + memset(pObj, 0, sizeof(SSyncRespMgr)); + + pObj->pRespHash = + taosHashInit(sizeof(uint64_t), taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); + assert(pObj->pRespHash != NULL); + pObj->ttl = ttl; + pObj->data = data; + pObj->seqNum = 0; + taosThreadMutexInit(&(pObj->mutex), NULL); + + return pObj; +} + +void syncRespMgrDestroy(SSyncRespMgr *pObj) { + taosThreadMutexLock(&(pObj->mutex)); + taosHashCleanup(pObj->pRespHash); + taosThreadMutexUnlock(&(pObj->mutex)); + taosThreadMutexDestroy(&(pObj->mutex)); + taosMemoryFree(pObj); +} + +int64_t syncRespMgrAdd(SSyncRespMgr *pObj, SRespStub *pStub) { + taosThreadMutexLock(&(pObj->mutex)); + + uint64_t keyCode = ++(pObj->seqNum); + taosHashPut(pObj->pRespHash, &keyCode, sizeof(keyCode), pStub, sizeof(SRespStub)); + + taosThreadMutexUnlock(&(pObj->mutex)); + return keyCode; +} + +int32_t syncRespMgrDel(SSyncRespMgr *pObj, uint64_t index) { + taosThreadMutexLock(&(pObj->mutex)); + + taosHashRemove(pObj->pRespHash, &index, sizeof(index)); + + taosThreadMutexUnlock(&(pObj->mutex)); + return 0; +} + +int32_t syncRespMgrGet(SSyncRespMgr *pObj, uint64_t index, SRespStub *pStub) { + taosThreadMutexLock(&(pObj->mutex)); + + void *pTmp = taosHashGet(pObj->pRespHash, &index, sizeof(index)); + if (pTmp != NULL) { + memcpy(pStub, pTmp, sizeof(SRespStub)); + taosThreadMutexUnlock(&(pObj->mutex)); + return 1; // get one object + } + taosThreadMutexUnlock(&(pObj->mutex)); + return 0; // get none object +} + +int32_t syncRespMgrGetAndDel(SSyncRespMgr *pObj, uint64_t index, SRespStub *pStub) { + taosThreadMutexLock(&(pObj->mutex)); + + void *pTmp = taosHashGet(pObj->pRespHash, &index, sizeof(index)); + if (pTmp != NULL) { + memcpy(pStub, pTmp, sizeof(SRespStub)); + taosThreadMutexUnlock(&(pObj->mutex)); + taosHashRemove(pObj->pRespHash, &index, sizeof(index)); + return 1; // get one object + } + taosThreadMutexUnlock(&(pObj->mutex)); + return 0; // get none object +} + +void syncRespClean(SSyncRespMgr *pObj) { + taosThreadMutexLock(&(pObj->mutex)); + syncRespCleanByTTL(pObj, pObj->ttl); + taosThreadMutexUnlock(&(pObj->mutex)); +} + +void syncRespCleanByTTL(SSyncRespMgr *pObj, int64_t ttl) {} \ No newline at end of file diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 287500a587..3b20dab0db 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -197,3 +197,17 @@ SyncIndex syncUtilMaxIndex(SyncIndex a, SyncIndex b) { SyncIndex r = a > b ? a : b; return r; } + +void syncUtilMsgHtoN(void* msg) { + // htonl + SMsgHead* pHead = msg; + pHead->contLen = htonl(pHead->contLen); + pHead->vgId = htonl(pHead->vgId); +} + +void syncUtilMsgNtoH(void* msg) { + // ntohl + SMsgHead* pHead = msg; + pHead->contLen = ntohl(pHead->contLen); + pHead->vgId = ntohl(pHead->vgId); +} \ No newline at end of file diff --git a/source/libs/sync/test/CMakeLists.txt b/source/libs/sync/test/CMakeLists.txt index 58b01e4ca2..85fe285661 100644 --- a/source/libs/sync/test/CMakeLists.txt +++ b/source/libs/sync/test/CMakeLists.txt @@ -38,6 +38,8 @@ add_executable(syncReplicateTest3 "") add_executable(syncReplicateLoadTest "") add_executable(syncRefTest "") add_executable(syncLogStoreCheck "") +add_executable(syncRaftCfgTest "") +add_executable(syncRespMgrTest "") target_sources(syncTest @@ -200,206 +202,224 @@ target_sources(syncLogStoreCheck PRIVATE "syncLogStoreCheck.cpp" ) +target_sources(syncRaftCfgTest + PRIVATE + "syncRaftCfgTest.cpp" +) +target_sources(syncRespMgrTest + PRIVATE + "syncRespMgrTest.cpp" +) target_include_directories(syncTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncEnvTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncPingTimerTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIOTickQTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIOTickPingTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIOSendMsgTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIOClientTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIOServerTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncRaftStoreTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncEnqTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIndexTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncInitTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncUtilTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncVotesGrantedTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncVotesRespondTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncIndexMgrTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncLogStoreTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncEntryTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncRequestVoteTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncRequestVoteReplyTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncAppendEntriesTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncAppendEntriesReplyTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncClientRequestTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncTimeoutTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncPingTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncPingReplyTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncRpcMsgTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncPingTimerTest2 PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncPingSelfTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncElectTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncElectTest2 PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncElectTest3 PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncEncodeTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncWriteTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncReplicateTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncReplicateTest2 PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncReplicateTest3 PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncReplicateLoadTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncRefTest PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) target_include_directories(syncLogStoreCheck PUBLIC - "${TD_SOURCE_DIR}/include/libs/sync" + "${CMAKE_SOURCE_DIR}/include/libs/sync" + "${CMAKE_CURRENT_SOURCE_DIR}/../inc" +) +target_include_directories(syncRaftCfgTest + PUBLIC + "${CMAKE_SOURCE_DIR}/include/libs/sync" + "${CMAKE_CURRENT_SOURCE_DIR}/../inc" +) +target_include_directories(syncRespMgrTest + PUBLIC + "${CMAKE_SOURCE_DIR}/include/libs/sync" "${CMAKE_CURRENT_SOURCE_DIR}/../inc" ) @@ -564,6 +584,14 @@ target_link_libraries(syncLogStoreCheck sync gtest_main ) +target_link_libraries(syncRaftCfgTest + sync + gtest_main +) +target_link_libraries(syncRespMgrTest + sync + gtest_main +) enable_testing() diff --git a/source/libs/sync/test/syncAppendEntriesReplyTest.cpp b/source/libs/sync/test/syncAppendEntriesReplyTest.cpp index 72aeb155e7..a90259bc3a 100644 --- a/source/libs/sync/test/syncAppendEntriesReplyTest.cpp +++ b/source/libs/sync/test/syncAppendEntriesReplyTest.cpp @@ -15,7 +15,7 @@ void logTest() { } SyncAppendEntriesReply *createMsg() { - SyncAppendEntriesReply *pMsg = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply *pMsg = syncAppendEntriesReplyBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -27,7 +27,7 @@ SyncAppendEntriesReply *createMsg() { void test1() { SyncAppendEntriesReply *pMsg = createMsg(); - syncAppendEntriesReplyPrint2((char *)"test1:", pMsg); + syncAppendEntriesReplyLog2((char *)"test1:", pMsg); syncAppendEntriesReplyDestroy(pMsg); } @@ -36,10 +36,10 @@ void test2() { uint32_t len = pMsg->bytes; char * serialized = (char *)taosMemoryMalloc(len); syncAppendEntriesReplySerialize(pMsg, serialized, len); - SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyBuild(1000); syncAppendEntriesReplyDeserialize(serialized, len, pMsg2); - syncAppendEntriesReplyPrint2((char *)"test2: syncAppendEntriesReplySerialize -> syncAppendEntriesReplyDeserialize ", - pMsg2); + syncAppendEntriesReplyLog2((char *)"test2: syncAppendEntriesReplySerialize -> syncAppendEntriesReplyDeserialize ", + pMsg2); taosMemoryFree(serialized); syncAppendEntriesReplyDestroy(pMsg); @@ -51,8 +51,8 @@ void test3() { uint32_t len; char * serialized = syncAppendEntriesReplySerialize2(pMsg, &len); SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyDeserialize2(serialized, len); - syncAppendEntriesReplyPrint2((char *)"test3: syncAppendEntriesReplySerialize3 -> syncAppendEntriesReplyDeserialize2 ", - pMsg2); + syncAppendEntriesReplyLog2((char *)"test3: syncAppendEntriesReplySerialize3 -> syncAppendEntriesReplyDeserialize2 ", + pMsg2); taosMemoryFree(serialized); syncAppendEntriesReplyDestroy(pMsg); @@ -63,11 +63,12 @@ void test4() { SyncAppendEntriesReply *pMsg = createMsg(); SRpcMsg rpcMsg; syncAppendEntriesReply2RpcMsg(pMsg, &rpcMsg); - SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyBuild(1000); syncAppendEntriesReplyFromRpcMsg(&rpcMsg, pMsg2); - syncAppendEntriesReplyPrint2((char *)"test4: syncAppendEntriesReply2RpcMsg -> syncAppendEntriesReplyFromRpcMsg ", - pMsg2); + syncAppendEntriesReplyLog2((char *)"test4: syncAppendEntriesReply2RpcMsg -> syncAppendEntriesReplyFromRpcMsg ", + pMsg2); + rpcFreeCont(rpcMsg.pCont); syncAppendEntriesReplyDestroy(pMsg); syncAppendEntriesReplyDestroy(pMsg2); } @@ -77,17 +78,17 @@ void test5() { SRpcMsg rpcMsg; syncAppendEntriesReply2RpcMsg(pMsg, &rpcMsg); SyncAppendEntriesReply *pMsg2 = syncAppendEntriesReplyFromRpcMsg2(&rpcMsg); - syncAppendEntriesReplyPrint2((char *)"test5: syncAppendEntriesReply2RpcMsg -> syncAppendEntriesReplyFromRpcMsg2 ", - pMsg2); + syncAppendEntriesReplyLog2((char *)"test5: syncAppendEntriesReply2RpcMsg -> syncAppendEntriesReplyFromRpcMsg2 ", + pMsg2); + rpcFreeCont(rpcMsg.pCont); syncAppendEntriesReplyDestroy(pMsg); syncAppendEntriesReplyDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncAppendEntriesTest.cpp b/source/libs/sync/test/syncAppendEntriesTest.cpp index 69a0aee9a8..bb9f306a1c 100644 --- a/source/libs/sync/test/syncAppendEntriesTest.cpp +++ b/source/libs/sync/test/syncAppendEntriesTest.cpp @@ -15,7 +15,7 @@ void logTest() { } SyncAppendEntries *createMsg() { - SyncAppendEntries *pMsg = syncAppendEntriesBuild(20); + SyncAppendEntries *pMsg = syncAppendEntriesBuild(20, 1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -29,7 +29,7 @@ SyncAppendEntries *createMsg() { void test1() { SyncAppendEntries *pMsg = createMsg(); - syncAppendEntriesPrint2((char *)"test1:", pMsg); + syncAppendEntriesLog2((char *)"test1:", pMsg); syncAppendEntriesDestroy(pMsg); } @@ -38,9 +38,9 @@ void test2() { uint32_t len = pMsg->bytes; char * serialized = (char *)taosMemoryMalloc(len); syncAppendEntriesSerialize(pMsg, serialized, len); - SyncAppendEntries *pMsg2 = syncAppendEntriesBuild(pMsg->dataLen); + SyncAppendEntries *pMsg2 = syncAppendEntriesBuild(pMsg->dataLen, 1000); syncAppendEntriesDeserialize(serialized, len, pMsg2); - syncAppendEntriesPrint2((char *)"test2: syncAppendEntriesSerialize -> syncAppendEntriesDeserialize ", pMsg2); + syncAppendEntriesLog2((char *)"test2: syncAppendEntriesSerialize -> syncAppendEntriesDeserialize ", pMsg2); taosMemoryFree(serialized); syncAppendEntriesDestroy(pMsg); @@ -52,7 +52,7 @@ void test3() { uint32_t len; char * serialized = syncAppendEntriesSerialize2(pMsg, &len); SyncAppendEntries *pMsg2 = syncAppendEntriesDeserialize2(serialized, len); - syncAppendEntriesPrint2((char *)"test3: syncAppendEntriesSerialize3 -> syncAppendEntriesDeserialize2 ", pMsg2); + syncAppendEntriesLog2((char *)"test3: syncAppendEntriesSerialize3 -> syncAppendEntriesDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncAppendEntriesDestroy(pMsg); @@ -65,8 +65,9 @@ void test4() { syncAppendEntries2RpcMsg(pMsg, &rpcMsg); SyncAppendEntries *pMsg2 = (SyncAppendEntries *)taosMemoryMalloc(rpcMsg.contLen); syncAppendEntriesFromRpcMsg(&rpcMsg, pMsg2); - syncAppendEntriesPrint2((char *)"test4: syncAppendEntries2RpcMsg -> syncAppendEntriesFromRpcMsg ", pMsg2); + syncAppendEntriesLog2((char *)"test4: syncAppendEntries2RpcMsg -> syncAppendEntriesFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncAppendEntriesDestroy(pMsg); syncAppendEntriesDestroy(pMsg2); } @@ -76,16 +77,16 @@ void test5() { SRpcMsg rpcMsg; syncAppendEntries2RpcMsg(pMsg, &rpcMsg); SyncAppendEntries *pMsg2 = syncAppendEntriesFromRpcMsg2(&rpcMsg); - syncAppendEntriesPrint2((char *)"test5: syncAppendEntries2RpcMsg -> syncAppendEntriesFromRpcMsg2 ", pMsg2); + syncAppendEntriesLog2((char *)"test5: syncAppendEntries2RpcMsg -> syncAppendEntriesFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncAppendEntriesDestroy(pMsg); syncAppendEntriesDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncClientRequestTest.cpp b/source/libs/sync/test/syncClientRequestTest.cpp index f22478d538..9376aca91f 100644 --- a/source/libs/sync/test/syncClientRequestTest.cpp +++ b/source/libs/sync/test/syncClientRequestTest.cpp @@ -21,13 +21,14 @@ SyncClientRequest *createMsg() { rpcMsg.contLen = 20; rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); strcpy((char *)rpcMsg.pCont, "hello rpc"); - SyncClientRequest *pMsg = syncClientRequestBuild2(&rpcMsg, 123, true); + SyncClientRequest *pMsg = syncClientRequestBuild2(&rpcMsg, 123, true, 1000); + rpcFreeCont(rpcMsg.pCont); return pMsg; } void test1() { SyncClientRequest *pMsg = createMsg(); - syncClientRequestPrint2((char *)"test1:", pMsg); + syncClientRequestLog2((char *)"test1:", pMsg); syncClientRequestDestroy(pMsg); } @@ -38,7 +39,7 @@ void test2() { syncClientRequestSerialize(pMsg, serialized, len); SyncClientRequest *pMsg2 = syncClientRequestBuild(pMsg->dataLen); syncClientRequestDeserialize(serialized, len, pMsg2); - syncClientRequestPrint2((char *)"test2: syncClientRequestSerialize -> syncClientRequestDeserialize ", pMsg2); + syncClientRequestLog2((char *)"test2: syncClientRequestSerialize -> syncClientRequestDeserialize ", pMsg2); taosMemoryFree(serialized); syncClientRequestDestroy(pMsg); @@ -50,7 +51,7 @@ void test3() { uint32_t len; char * serialized = syncClientRequestSerialize2(pMsg, &len); SyncClientRequest *pMsg2 = syncClientRequestDeserialize2(serialized, len); - syncClientRequestPrint2((char *)"test3: syncClientRequestSerialize3 -> syncClientRequestDeserialize2 ", pMsg2); + syncClientRequestLog2((char *)"test3: syncClientRequestSerialize3 -> syncClientRequestDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncClientRequestDestroy(pMsg); @@ -63,8 +64,9 @@ void test4() { syncClientRequest2RpcMsg(pMsg, &rpcMsg); SyncClientRequest *pMsg2 = (SyncClientRequest *)taosMemoryMalloc(rpcMsg.contLen); syncClientRequestFromRpcMsg(&rpcMsg, pMsg2); - syncClientRequestPrint2((char *)"test4: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg ", pMsg2); + syncClientRequestLog2((char *)"test4: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncClientRequestDestroy(pMsg); syncClientRequestDestroy(pMsg2); } @@ -74,16 +76,16 @@ void test5() { SRpcMsg rpcMsg; syncClientRequest2RpcMsg(pMsg, &rpcMsg); SyncClientRequest *pMsg2 = syncClientRequestFromRpcMsg2(&rpcMsg); - syncClientRequestPrint2((char *)"test5: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg2 ", pMsg2); + syncClientRequestLog2((char *)"test5: syncClientRequest2RpcMsg -> syncClientRequestFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncClientRequestDestroy(pMsg); syncClientRequestDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncElectTest.cpp b/source/libs/sync/test/syncElectTest.cpp index 251a4b538f..39b5eba1aa 100644 --- a/source/libs/sync/test/syncElectTest.cpp +++ b/source/libs/sync/test/syncElectTest.cpp @@ -20,11 +20,12 @@ uint16_t ports[] = {7010, 7110, 7210, 7310, 7410}; int32_t replicaNum = 3; int32_t myIndex = 0; -SRaftId ids[TSDB_MAX_REPLICA]; -SSyncInfo syncInfo; -SSyncFSM* pFsm; -SWal* pWal; -SSyncNode* gSyncNode; +SRaftId ids[TSDB_MAX_REPLICA]; +SSyncInfo syncInfo; +SSyncFSM* pFsm; +SWal* pWal; +SSyncNode* gSyncNode; +const char* pDir = "./syncElectTest"; SSyncNode* syncNodeInit() { syncInfo.vgId = 1234; @@ -33,7 +34,7 @@ SSyncNode* syncNodeInit() { syncInfo.queue = gSyncIO->pMsgQ; syncInfo.FpEqMsg = syncIOEqMsg; syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "./elect_test_%d", myIndex); + snprintf(syncInfo.path, sizeof(syncInfo.path), "%s_sync_%d", pDir, myIndex); int code = walInit(); assert(code == 0); @@ -48,7 +49,7 @@ SSyncNode* syncNodeInit() { walCfg.level = TAOS_WAL_FSYNC; char tmpdir[128]; - snprintf(tmpdir, sizeof(tmpdir), "./elect_test_wal_%d", myIndex); + snprintf(tmpdir, sizeof(tmpdir), "%s_wal_%d", pDir, myIndex); pWal = walOpen(tmpdir, &walCfg); assert(pWal != NULL); @@ -78,6 +79,8 @@ SSyncNode* syncNodeInit() { gSyncIO->FpOnSyncTimeout = pSyncNode->FpOnTimeout; gSyncIO->pSyncNode = pSyncNode; + syncNodeStart(pSyncNode); + return pSyncNode; } @@ -110,16 +113,15 @@ int main(int argc, char** argv) { gSyncNode = syncInitTest(); assert(gSyncNode != NULL); - syncNodePrint2((char*)"", gSyncNode); + syncNodeLog2((char*)"", gSyncNode); initRaftId(gSyncNode); //--------------------------- while (1) { - sTrace( - "elect sleep, state: %d, %s, term:%lu electTimerLogicClock:%lu, electTimerLogicClockUser:%lu, electTimerMS:%d", - gSyncNode->state, syncUtilState2String(gSyncNode->state), gSyncNode->pRaftStore->currentTerm, - gSyncNode->electTimerLogicClock, gSyncNode->electTimerLogicClockUser, gSyncNode->electTimerMS); + char* s = syncNode2SimpleStr(gSyncNode); + sTrace("%s", s); + taosMemoryFree(s); taosMsleep(1000); } diff --git a/source/libs/sync/test/syncElectTest2.cpp b/source/libs/sync/test/syncElectTest2.cpp index 709d9d8580..d569b47fb1 100644 --- a/source/libs/sync/test/syncElectTest2.cpp +++ b/source/libs/sync/test/syncElectTest2.cpp @@ -21,10 +21,11 @@ uint16_t ports[] = {7010, 7110, 7210, 7310, 7410}; int32_t replicaNum = 3; int32_t myIndex = 0; -SRaftId ids[TSDB_MAX_REPLICA]; -SSyncInfo syncInfo; -SSyncFSM* pFsm; -SWal* pWal; +SRaftId ids[TSDB_MAX_REPLICA]; +SSyncInfo syncInfo; +SSyncFSM* pFsm; +SWal* pWal; +const char* pDir = "./syncElectTest2"; int64_t syncNodeInit() { syncInfo.vgId = 1234; @@ -33,7 +34,7 @@ int64_t syncNodeInit() { syncInfo.queue = gSyncIO->pMsgQ; syncInfo.FpEqMsg = syncIOEqMsg; syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "./elect2_test_%d", myIndex); + snprintf(syncInfo.path, sizeof(syncInfo.path), "%s_sync_%d", pDir, myIndex); int code = walInit(); assert(code == 0); @@ -48,7 +49,7 @@ int64_t syncNodeInit() { walCfg.level = TAOS_WAL_FSYNC; char tmpdir[128]; - snprintf(tmpdir, sizeof(tmpdir), "./elect2_test_wal_%d", myIndex); + snprintf(tmpdir, sizeof(tmpdir), "%s_wal_%d", pDir, myIndex); pWal = walOpen(tmpdir, &walCfg); assert(pWal != NULL); @@ -64,8 +65,9 @@ int64_t syncNodeInit() { // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); } - int64_t rid = syncStart(&syncInfo); + int64_t rid = syncOpen(&syncInfo); assert(rid > 0); + syncStart(rid); SSyncNode* pSyncNode = (SSyncNode*)syncNodeAcquire(rid); assert(pSyncNode != NULL); @@ -81,6 +83,8 @@ int64_t syncNodeInit() { gSyncIO->FpOnSyncTimeout = pSyncNode->FpOnTimeout; gSyncIO->pSyncNode = pSyncNode; + syncNodeStart(pSyncNode); + syncNodeRelease(pSyncNode); return rid; @@ -117,7 +121,7 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = (SSyncNode*)syncNodeAcquire(rid); assert(pSyncNode != NULL); - syncNodePrint2((char*)"", pSyncNode); + syncNodeLog2((char*)"", pSyncNode); initRaftId(pSyncNode); //--------------------------- diff --git a/source/libs/sync/test/syncElectTest3.cpp b/source/libs/sync/test/syncElectTest3.cpp index e36a2d0fb1..59e1b993d8 100644 --- a/source/libs/sync/test/syncElectTest3.cpp +++ b/source/libs/sync/test/syncElectTest3.cpp @@ -21,10 +21,11 @@ uint16_t ports[] = {7010, 7110, 7210, 7310, 7410}; int32_t replicaNum = 3; int32_t myIndex = 0; -SRaftId ids[TSDB_MAX_REPLICA]; -SSyncInfo syncInfo; -SSyncFSM* pFsm; -SWal* pWal; +SRaftId ids[TSDB_MAX_REPLICA]; +SSyncInfo syncInfo; +SSyncFSM* pFsm; +SWal* pWal; +const char* pDir = "./syncElectTest3"; int64_t syncNodeInit() { syncInfo.vgId = 1234; @@ -33,7 +34,7 @@ int64_t syncNodeInit() { syncInfo.queue = gSyncIO->pMsgQ; syncInfo.FpEqMsg = syncIOEqMsg; syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "./elect3_test_%d", myIndex); + snprintf(syncInfo.path, sizeof(syncInfo.path), "%s_sync_%d", pDir, myIndex); int code = walInit(); assert(code == 0); @@ -48,7 +49,7 @@ int64_t syncNodeInit() { walCfg.level = TAOS_WAL_FSYNC; char tmpdir[128]; - snprintf(tmpdir, sizeof(tmpdir), "./elect3_test_wal_%d", myIndex); + snprintf(tmpdir, sizeof(tmpdir), "%s_wal_%d", pDir, myIndex); pWal = walOpen(tmpdir, &walCfg); assert(pWal != NULL); @@ -64,8 +65,9 @@ int64_t syncNodeInit() { // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); } - int64_t rid = syncStart(&syncInfo); + int64_t rid = syncOpen(&syncInfo); assert(rid > 0); + syncStart(rid); SSyncNode* pSyncNode = (SSyncNode*)syncNodeAcquire(rid); assert(pSyncNode != NULL); @@ -84,6 +86,8 @@ int64_t syncNodeInit() { gSyncIO->FpOnSyncTimeout = pSyncNode->FpOnTimeout; gSyncIO->pSyncNode = pSyncNode; + syncNodeStart(pSyncNode); + syncNodeRelease(pSyncNode); return rid; @@ -120,7 +124,7 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = (SSyncNode*)syncNodeAcquire(rid); assert(pSyncNode != NULL); - syncNodePrint2((char*)"", pSyncNode); + syncNodeLog2((char*)"", pSyncNode); initRaftId(pSyncNode); //--------------------------- diff --git a/source/libs/sync/test/syncEncodeTest.cpp b/source/libs/sync/test/syncEncodeTest.cpp index b1ba95dc3e..f052d08a1e 100644 --- a/source/libs/sync/test/syncEncodeTest.cpp +++ b/source/libs/sync/test/syncEncodeTest.cpp @@ -102,7 +102,7 @@ SRpcMsg *step0() { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -159,23 +159,23 @@ int main(int argc, char **argv) { // step0 SRpcMsg *pMsg0 = step0(); - syncRpcMsgPrint2((char *)"==step0==", pMsg0); + syncRpcMsgLog2((char *)"==step0==", pMsg0); // step1 SyncClientRequest *pMsg1 = step1(pMsg0); - syncClientRequestPrint2((char *)"==step1==", pMsg1); + syncClientRequestLog2((char *)"==step1==", pMsg1); // step2 SRpcMsg *pMsg2 = step2(pMsg1); - syncRpcMsgPrint2((char *)"==step2==", pMsg2); + syncRpcMsgLog2((char *)"==step2==", pMsg2); // step3 SyncClientRequest *pMsg3 = step3(pMsg2); - syncClientRequestPrint2((char *)"==step3==", pMsg3); + syncClientRequestLog2((char *)"==step3==", pMsg3); // step4 SSyncRaftEntry *pMsg4 = step4(pMsg3); - syncEntryPrint2((char *)"==step4==", pMsg4); + syncEntryLog2((char *)"==step4==", pMsg4); // log, relog SSyncNode *pSyncNode = syncNodeInit(); @@ -183,7 +183,7 @@ int main(int argc, char **argv) { SSyncRaftEntry *pEntry = pMsg4; pSyncNode->pLogStore->appendEntry(pSyncNode->pLogStore, pEntry); SSyncRaftEntry *pEntry2 = pSyncNode->pLogStore->getEntry(pSyncNode->pLogStore, pEntry->index); - syncEntryPrint2((char *)"==pEntry2==", pEntry2); + syncEntryLog2((char *)"==pEntry2==", pEntry2); // step5 uint32_t len; @@ -194,11 +194,11 @@ int main(int argc, char **argv) { // step6 SSyncRaftEntry *pMsg6 = step6(pMsg5, len); - syncEntryPrint2((char *)"==step6==", pMsg6); + syncEntryLog2((char *)"==step6==", pMsg6); // step7 SRpcMsg *pMsg7 = step7(pMsg6); - syncRpcMsgPrint2((char *)"==step7==", pMsg7); + syncRpcMsgLog2((char *)"==step7==", pMsg7); return 0; } diff --git a/source/libs/sync/test/syncEnqTest.cpp b/source/libs/sync/test/syncEnqTest.cpp index 48cc7caf2f..6f83ede5a0 100644 --- a/source/libs/sync/test/syncEnqTest.cpp +++ b/source/libs/sync/test/syncEnqTest.cpp @@ -89,14 +89,14 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"syncInitTest", pSyncNode); + syncNodeLog2((char*)"syncInitTest", pSyncNode); initRaftId(pSyncNode); //-------------------------------------------------------------- for (int i = 0; i < 10; ++i) { - SyncPingReply* pSyncMsg = syncPingReplyBuild2(&pSyncNode->myRaftId, &pSyncNode->myRaftId, "syncEnqTest"); + SyncPingReply* pSyncMsg = syncPingReplyBuild2(&pSyncNode->myRaftId, &pSyncNode->myRaftId, 1000, "syncEnqTest"); SRpcMsg rpcMsg; syncPingReply2RpcMsg(pSyncMsg, &rpcMsg); pSyncNode->FpEqMsg(pSyncNode->queue, &rpcMsg); diff --git a/source/libs/sync/test/syncEntryTest.cpp b/source/libs/sync/test/syncEntryTest.cpp index e8427e8168..53ae91fc60 100644 --- a/source/libs/sync/test/syncEntryTest.cpp +++ b/source/libs/sync/test/syncEntryTest.cpp @@ -52,7 +52,7 @@ void test3() { pSyncMsg->isWeak = 1; strcpy(pSyncMsg->data, "test3"); - SSyncRaftEntry* pEntry = syncEntryBuild3(pSyncMsg, 100, 200, SYNC_RAFT_ENTRY_NOOP); + SSyncRaftEntry* pEntry = syncEntryBuild3(pSyncMsg, 100, 200); syncEntryPrint(pEntry); syncClientRequestDestroy(pSyncMsg); @@ -68,7 +68,6 @@ void test4() { pEntry->isWeak = true; pEntry->term = 44; pEntry->index = 55; - pEntry->entryType = SYNC_RAFT_ENTRY_CONFIG; strcpy(pEntry->data, "test4"); syncEntryPrint(pEntry); @@ -84,9 +83,8 @@ void test4() { } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; test1(); test2(); diff --git a/source/libs/sync/test/syncIOClientTest.cpp b/source/libs/sync/test/syncIOClientTest.cpp index dffa8b5cb9..c146ff4e92 100644 --- a/source/libs/sync/test/syncIOClientTest.cpp +++ b/source/libs/sync/test/syncIOClientTest.cpp @@ -38,7 +38,7 @@ int main() { destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); destId.vgId = 100; - SyncPingReply* pSyncMsg = syncPingReplyBuild2(&srcId, &destId, "syncIOClientTest"); + SyncPingReply* pSyncMsg = syncPingReplyBuild2(&srcId, &destId, 1000, "syncIOClientTest"); SRpcMsg rpcMsg; syncPingReply2RpcMsg(pSyncMsg, &rpcMsg); diff --git a/source/libs/sync/test/syncIOSendMsgTest.cpp b/source/libs/sync/test/syncIOSendMsgTest.cpp index 913e57e59a..03d308ea28 100644 --- a/source/libs/sync/test/syncIOSendMsgTest.cpp +++ b/source/libs/sync/test/syncIOSendMsgTest.cpp @@ -89,15 +89,16 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"syncInitTest", pSyncNode); + syncNodeLog2((char*)"syncInitTest", pSyncNode); initRaftId(pSyncNode); //-------------------------------------------------------------- for (int i = 0; i < 10; ++i) { - SyncPingReply* pSyncMsg = syncPingReplyBuild2(&pSyncNode->myRaftId, &pSyncNode->myRaftId, "syncIOSendMsgTest"); - SRpcMsg rpcMsg; + SyncPingReply* pSyncMsg = + syncPingReplyBuild2(&pSyncNode->myRaftId, &pSyncNode->myRaftId, 1000, "syncIOSendMsgTest"); + SRpcMsg rpcMsg; syncPingReply2RpcMsg(pSyncMsg, &rpcMsg); SEpSet epSet; diff --git a/source/libs/sync/test/syncIndexMgrTest.cpp b/source/libs/sync/test/syncIndexMgrTest.cpp index 319ea3e15a..ea5d5f6b6f 100644 --- a/source/libs/sync/test/syncIndexMgrTest.cpp +++ b/source/libs/sync/test/syncIndexMgrTest.cpp @@ -73,9 +73,8 @@ void initRaftId(SSyncNode* pSyncNode) { } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; myIndex = 0; if (argc >= 2) { diff --git a/source/libs/sync/test/syncInitTest.cpp b/source/libs/sync/test/syncInitTest.cpp index 48b5488e41..ca0657c74d 100644 --- a/source/libs/sync/test/syncInitTest.cpp +++ b/source/libs/sync/test/syncInitTest.cpp @@ -30,7 +30,7 @@ SSyncNode* syncNodeInit() { syncInfo.queue = gSyncIO->pMsgQ; syncInfo.FpEqMsg = syncIOEqMsg; syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "%s", "./"); + snprintf(syncInfo.path, sizeof(syncInfo.path), "%s", "./sync_init_test"); SSyncCfg* pCfg = &syncInfo.syncCfg; pCfg->myIndex = myIndex; @@ -64,15 +64,15 @@ void initRaftId(SSyncNode* pSyncNode) { for (int i = 0; i < replicaNum; ++i) { ids[i] = pSyncNode->replicasId[i]; char* s = syncUtilRaftId2Str(&ids[i]); - printf("raftId[%d] : %s\n", i, s); + sTrace("raftId[%d] : %s\n", i, s); taosMemoryFree(s); } } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); + // taosInitLog((char *)"tmp/syncInitTest.log", 100); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; myIndex = 0; if (argc >= 2) { @@ -88,11 +88,13 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"syncInitTest", pSyncNode); - + syncNodeLog2((char*)"syncInitTest", pSyncNode); initRaftId(pSyncNode); - //-------------------------------------------------------------- + syncNodeClose(pSyncNode); + syncEnvStop(); + // syncIOStop(); + // taosCloseLog(); return 0; } \ No newline at end of file diff --git a/source/libs/sync/test/syncLogStoreCheck.cpp b/source/libs/sync/test/syncLogStoreCheck.cpp index 85e39a61c5..b77a744c30 100644 --- a/source/libs/sync/test/syncLogStoreCheck.cpp +++ b/source/libs/sync/test/syncLogStoreCheck.cpp @@ -99,7 +99,7 @@ int main(int argc, char** argv) { pSyncNode = logStoreCheck(argv[1]); assert(pSyncNode != NULL); - logStorePrint2((char*)"logStoreCheck", pSyncNode->pLogStore); + logStoreLog2((char*)"logStoreCheck", pSyncNode->pLogStore); return 0; } diff --git a/source/libs/sync/test/syncLogStoreTest.cpp b/source/libs/sync/test/syncLogStoreTest.cpp index a5adba9a88..c4a5f6cae4 100644 --- a/source/libs/sync/test/syncLogStoreTest.cpp +++ b/source/libs/sync/test/syncLogStoreTest.cpp @@ -16,74 +16,44 @@ void logTest() { sFatal("--- sync log test: fatal"); } -uint16_t ports[] = {7010, 7110, 7210, 7310, 7410}; -int32_t replicaNum = 1; -int32_t myIndex = 0; +SSyncNode* pSyncNode; +SWal* pWal; +SSyncLogStore* pLogStore; +const char* pWalPath = "./syncLogStoreTest_wal"; -SRaftId ids[TSDB_MAX_REPLICA]; -SSyncInfo syncInfo; -SSyncFSM* pFsm; -SWal* pWal; -SSyncNode* pSyncNode; +void init() { + walInit(); + taosRemoveDir(pWalPath); -SSyncNode* syncNodeInit() { - syncInfo.vgId = 1234; - syncInfo.rpcClient = gSyncIO->clientRpc; - syncInfo.FpSendMsg = syncIOSendMsg; - syncInfo.queue = gSyncIO->pMsgQ; - syncInfo.FpEqMsg = syncIOEqMsg; - syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "%s", "./"); - - int code = walInit(); - assert(code == 0); SWalCfg walCfg; memset(&walCfg, 0, sizeof(SWalCfg)); - walCfg.vgId = syncInfo.vgId; + walCfg.vgId = 1000; walCfg.fsyncPeriod = 1000; walCfg.retentionPeriod = 1000; walCfg.rollPeriod = 1000; walCfg.retentionSize = 1000; walCfg.segSize = 1000; walCfg.level = TAOS_WAL_FSYNC; - pWal = walOpen("./wal_test", &walCfg); + pWal = walOpen(pWalPath, &walCfg); assert(pWal != NULL); - syncInfo.pWal = pWal; - - SSyncCfg* pCfg = &syncInfo.syncCfg; - pCfg->myIndex = myIndex; - pCfg->replicaNum = replicaNum; - - for (int i = 0; i < replicaNum; ++i) { - pCfg->nodeInfo[i].nodePort = ports[i]; - snprintf(pCfg->nodeInfo[i].nodeFqdn, sizeof(pCfg->nodeInfo[i].nodeFqdn), "%s", "127.0.0.1"); - // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); - } - - pSyncNode = syncNodeOpen(&syncInfo); - assert(pSyncNode != NULL); - - gSyncIO->FpOnSyncPing = pSyncNode->FpOnPing; - gSyncIO->FpOnSyncPingReply = pSyncNode->FpOnPingReply; - gSyncIO->FpOnSyncRequestVote = pSyncNode->FpOnRequestVote; - gSyncIO->FpOnSyncRequestVoteReply = pSyncNode->FpOnRequestVoteReply; - gSyncIO->FpOnSyncAppendEntries = pSyncNode->FpOnAppendEntries; - gSyncIO->FpOnSyncAppendEntriesReply = pSyncNode->FpOnAppendEntriesReply; - gSyncIO->FpOnSyncPing = pSyncNode->FpOnPing; - gSyncIO->FpOnSyncPingReply = pSyncNode->FpOnPingReply; - gSyncIO->FpOnSyncTimeout = pSyncNode->FpOnTimeout; - gSyncIO->pSyncNode = pSyncNode; - - return pSyncNode; + pSyncNode = (SSyncNode*)taosMemoryMalloc(sizeof(SSyncNode)); + memset(pSyncNode, 0, sizeof(SSyncNode)); + pSyncNode->pWal = pWal; } -SSyncNode* syncInitTest() { return syncNodeInit(); } +void cleanup() { + walClose(pWal); + walCleanUp(); + taosMemoryFree(pSyncNode); +} void logStoreTest() { - logStorePrint2((char*)"logStoreTest", pSyncNode->pLogStore); + pLogStore = logStoreCreate(pSyncNode); + assert(pLogStore); + assert(pLogStore->getLastIndex(pLogStore) == SYNC_INDEX_INVALID); - assert(pSyncNode->pLogStore->getLastIndex(pSyncNode->pLogStore) == SYNC_INDEX_INVALID); + logStoreLog2((char*)"logStoreTest", pLogStore); for (int i = 0; i < 5; ++i) { int32_t dataLen = 10; @@ -93,58 +63,35 @@ void logStoreTest() { pEntry->originalRpcType = 2; pEntry->seqNum = 3; pEntry->isWeak = true; - pEntry->term = 100; - pEntry->index = pSyncNode->pLogStore->getLastIndex(pSyncNode->pLogStore) + 1; + pEntry->term = 100 + i; + pEntry->index = pLogStore->getLastIndex(pLogStore) + 1; snprintf(pEntry->data, dataLen, "value%d", i); - // syncEntryPrint2((char*)"write entry:", pEntry); - pSyncNode->pLogStore->appendEntry(pSyncNode->pLogStore, pEntry); + syncEntryLog2((char*)"==write entry== :", pEntry); + pLogStore->appendEntry(pLogStore, pEntry); syncEntryDestory(pEntry); if (i == 0) { - assert(pSyncNode->pLogStore->getLastIndex(pSyncNode->pLogStore) == SYNC_INDEX_BEGIN); + assert(pLogStore->getLastIndex(pLogStore) == SYNC_INDEX_BEGIN); } } - logStorePrint2((char*)"after appendEntry", pSyncNode->pLogStore); + logStoreLog2((char*)"after appendEntry", pLogStore); - pSyncNode->pLogStore->truncate(pSyncNode->pLogStore, 3); - logStorePrint2((char*)"after truncate 3", pSyncNode->pLogStore); -} + pLogStore->truncate(pLogStore, 3); + logStoreLog2((char*)"after truncate 3", pLogStore); -void initRaftId(SSyncNode* pSyncNode) { - for (int i = 0; i < replicaNum; ++i) { - ids[i] = pSyncNode->replicasId[i]; - char* s = syncUtilRaftId2Str(&ids[i]); - printf("raftId[%d] : %s\n", i, s); - taosMemoryFree(s); - } + logStoreDestory(pLogStore); } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; - - myIndex = 0; - if (argc >= 2) { - myIndex = atoi(argv[1]); - } - - int32_t ret = syncIOStart((char*)"127.0.0.1", ports[myIndex]); - assert(ret == 0); - - ret = syncEnvStart(); - assert(ret == 0); - - taosRemoveDir("./wal_test"); - - pSyncNode = syncInitTest(); - assert(pSyncNode != NULL); - - // syncNodePrint((char*)"syncLogStoreTest", pSyncNode); - // initRaftId(pSyncNode); + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; + init(); logStoreTest(); + taosMsleep(2000); + cleanup(); + return 0; } diff --git a/source/libs/sync/test/syncPingReplyTest.cpp b/source/libs/sync/test/syncPingReplyTest.cpp index 40592eab6f..2bd8671e6c 100644 --- a/source/libs/sync/test/syncPingReplyTest.cpp +++ b/source/libs/sync/test/syncPingReplyTest.cpp @@ -20,13 +20,13 @@ SyncPingReply *createMsg() { srcId.vgId = 100; destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); destId.vgId = 100; - SyncPingReply *pMsg = syncPingReplyBuild3(&srcId, &destId); + SyncPingReply *pMsg = syncPingReplyBuild3(&srcId, &destId, 1000); return pMsg; } void test1() { SyncPingReply *pMsg = createMsg(); - syncPingReplyPrint2((char *)"test1:", pMsg); + syncPingReplyLog2((char *)"test1:", pMsg); syncPingReplyDestroy(pMsg); } @@ -37,7 +37,7 @@ void test2() { syncPingReplySerialize(pMsg, serialized, len); SyncPingReply *pMsg2 = syncPingReplyBuild(pMsg->dataLen); syncPingReplyDeserialize(serialized, len, pMsg2); - syncPingReplyPrint2((char *)"test2: syncPingReplySerialize -> syncPingReplyDeserialize ", pMsg2); + syncPingReplyLog2((char *)"test2: syncPingReplySerialize -> syncPingReplyDeserialize ", pMsg2); taosMemoryFree(serialized); syncPingReplyDestroy(pMsg); @@ -49,7 +49,7 @@ void test3() { uint32_t len; char * serialized = syncPingReplySerialize2(pMsg, &len); SyncPingReply *pMsg2 = syncPingReplyDeserialize2(serialized, len); - syncPingReplyPrint2((char *)"test3: syncPingReplySerialize3 -> syncPingReplyDeserialize2 ", pMsg2); + syncPingReplyLog2((char *)"test3: syncPingReplySerialize2 -> syncPingReplyDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncPingReplyDestroy(pMsg); @@ -62,8 +62,9 @@ void test4() { syncPingReply2RpcMsg(pMsg, &rpcMsg); SyncPingReply *pMsg2 = (SyncPingReply *)taosMemoryMalloc(rpcMsg.contLen); syncPingReplyFromRpcMsg(&rpcMsg, pMsg2); - syncPingReplyPrint2((char *)"test4: syncPingReply2RpcMsg -> syncPingReplyFromRpcMsg ", pMsg2); + syncPingReplyLog2((char *)"test4: syncPingReply2RpcMsg -> syncPingReplyFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncPingReplyDestroy(pMsg); syncPingReplyDestroy(pMsg2); } @@ -73,16 +74,30 @@ void test5() { SRpcMsg rpcMsg; syncPingReply2RpcMsg(pMsg, &rpcMsg); SyncPingReply *pMsg2 = syncPingReplyFromRpcMsg2(&rpcMsg); - syncPingReplyPrint2((char *)"test5: syncPingReply2RpcMsg -> syncPingReplyFromRpcMsg2 ", pMsg2); + syncPingReplyLog2((char *)"test5: syncPingReply2RpcMsg -> syncPingReplyFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); + syncPingReplyDestroy(pMsg); + syncPingReplyDestroy(pMsg2); +} + +void test6() { + SyncPingReply *pMsg = createMsg(); + int32_t bufLen = syncPingReplySerialize3(pMsg, NULL, 0); + char * serialized = (char *)taosMemoryMalloc(bufLen); + syncPingReplySerialize3(pMsg, serialized, bufLen); + SyncPingReply *pMsg2 = syncPingReplyDeserialize3(serialized, bufLen); + assert(pMsg2 != NULL); + syncPingReplyLog2((char *)"test6: syncPingReplySerialize3 -> syncPingReplyDeserialize3 ", pMsg2); + + taosMemoryFree(serialized); syncPingReplyDestroy(pMsg); syncPingReplyDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); @@ -90,6 +105,7 @@ int main() { test3(); test4(); test5(); + test6(); return 0; } diff --git a/source/libs/sync/test/syncPingSelfTest.cpp b/source/libs/sync/test/syncPingSelfTest.cpp index 677518c6ac..641ff059be 100644 --- a/source/libs/sync/test/syncPingSelfTest.cpp +++ b/source/libs/sync/test/syncPingSelfTest.cpp @@ -87,7 +87,7 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"", pSyncNode); + syncNodeLog2((char*)"", pSyncNode); initRaftId(pSyncNode); diff --git a/source/libs/sync/test/syncPingTest.cpp b/source/libs/sync/test/syncPingTest.cpp index eb774d77c3..a02f5e4b43 100644 --- a/source/libs/sync/test/syncPingTest.cpp +++ b/source/libs/sync/test/syncPingTest.cpp @@ -20,13 +20,13 @@ SyncPing *createMsg() { srcId.vgId = 100; destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); destId.vgId = 100; - SyncPing *pMsg = syncPingBuild3(&srcId, &destId); + SyncPing *pMsg = syncPingBuild3(&srcId, &destId, 1000); return pMsg; } void test1() { SyncPing *pMsg = createMsg(); - syncPingPrint2((char *)"test1:", pMsg); + syncPingLog2((char *)"test1:", pMsg); syncPingDestroy(pMsg); } @@ -37,7 +37,7 @@ void test2() { syncPingSerialize(pMsg, serialized, len); SyncPing *pMsg2 = syncPingBuild(pMsg->dataLen); syncPingDeserialize(serialized, len, pMsg2); - syncPingPrint2((char *)"test2: syncPingSerialize -> syncPingDeserialize ", pMsg2); + syncPingLog2((char *)"test2: syncPingSerialize -> syncPingDeserialize ", pMsg2); taosMemoryFree(serialized); syncPingDestroy(pMsg); @@ -49,7 +49,7 @@ void test3() { uint32_t len; char * serialized = syncPingSerialize2(pMsg, &len); SyncPing *pMsg2 = syncPingDeserialize2(serialized, len); - syncPingPrint2((char *)"test3: syncPingSerialize3 -> syncPingDeserialize2 ", pMsg2); + syncPingLog2((char *)"test3: syncPingSerialize2 -> syncPingDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncPingDestroy(pMsg); @@ -62,10 +62,11 @@ void test4() { syncPing2RpcMsg(pMsg, &rpcMsg); SyncPing *pMsg2 = (SyncPing *)taosMemoryMalloc(rpcMsg.contLen); syncPingFromRpcMsg(&rpcMsg, pMsg2); - syncPingPrint2((char *)"test4: syncPing2RpcMsg -> syncPingFromRpcMsg ", pMsg2); + syncPingLog2((char *)"test4: syncPing2RpcMsg -> syncPingFromRpcMsg ", pMsg2); syncPingDestroy(pMsg); syncPingDestroy(pMsg2); + rpcFreeCont(rpcMsg.pCont); } void test5() { @@ -73,16 +74,30 @@ void test5() { SRpcMsg rpcMsg; syncPing2RpcMsg(pMsg, &rpcMsg); SyncPing *pMsg2 = syncPingFromRpcMsg2(&rpcMsg); - syncPingPrint2((char *)"test5: syncPing2RpcMsg -> syncPingFromRpcMsg2 ", pMsg2); + syncPingLog2((char *)"test5: syncPing2RpcMsg -> syncPingFromRpcMsg2 ", pMsg2); syncPingDestroy(pMsg); syncPingDestroy(pMsg2); + rpcFreeCont(rpcMsg.pCont); +} + +void test6() { + SyncPing *pMsg = createMsg(); + int32_t bufLen = syncPingSerialize3(pMsg, NULL, 0); + char * serialized = (char *)taosMemoryMalloc(bufLen); + syncPingSerialize3(pMsg, serialized, bufLen); + SyncPing *pMsg2 = syncPingDeserialize3(serialized, bufLen); + assert(pMsg2 != NULL); + syncPingLog2((char *)"test6: syncPingSerialize3 -> syncPingDeserialize3 ", pMsg2); + + taosMemoryFree(serialized); + syncPingDestroy(pMsg); + syncPingDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); @@ -90,6 +105,7 @@ int main() { test3(); test4(); test5(); + test6(); return 0; } diff --git a/source/libs/sync/test/syncPingTimerTest.cpp b/source/libs/sync/test/syncPingTimerTest.cpp index 3c5e76ca14..29e99435be 100644 --- a/source/libs/sync/test/syncPingTimerTest.cpp +++ b/source/libs/sync/test/syncPingTimerTest.cpp @@ -87,7 +87,7 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"----1", pSyncNode); + syncNodeLog2((char*)"----1", pSyncNode); initRaftId(pSyncNode); @@ -96,7 +96,7 @@ int main(int argc, char** argv) { sTrace("syncNodeStartPingTimer ..."); ret = syncNodeStartPingTimer(pSyncNode); assert(ret == 0); - syncNodePrint2((char*)"----2", pSyncNode); + syncNodeLog2((char*)"----2", pSyncNode); sTrace("sleep ..."); taosMsleep(10000); @@ -104,7 +104,7 @@ int main(int argc, char** argv) { sTrace("syncNodeStopPingTimer ..."); ret = syncNodeStopPingTimer(pSyncNode); assert(ret == 0); - syncNodePrint2((char*)"----3", pSyncNode); + syncNodeLog2((char*)"----3", pSyncNode); sTrace("sleep ..."); taosMsleep(5000); @@ -112,7 +112,7 @@ int main(int argc, char** argv) { sTrace("syncNodeStartPingTimer ..."); ret = syncNodeStartPingTimer(pSyncNode); assert(ret == 0); - syncNodePrint2((char*)"----4", pSyncNode); + syncNodeLog2((char*)"----4", pSyncNode); sTrace("sleep ..."); taosMsleep(10000); @@ -120,7 +120,7 @@ int main(int argc, char** argv) { sTrace("syncNodeStopPingTimer ..."); ret = syncNodeStopPingTimer(pSyncNode); assert(ret == 0); - syncNodePrint2((char*)"----5", pSyncNode); + syncNodeLog2((char*)"----5", pSyncNode); while (1) { sTrace("while 1 sleep ..."); diff --git a/source/libs/sync/test/syncPingTimerTest2.cpp b/source/libs/sync/test/syncPingTimerTest2.cpp index 554f67d365..285828125d 100644 --- a/source/libs/sync/test/syncPingTimerTest2.cpp +++ b/source/libs/sync/test/syncPingTimerTest2.cpp @@ -87,7 +87,7 @@ int main(int argc, char** argv) { SSyncNode* pSyncNode = syncInitTest(); assert(pSyncNode != NULL); - syncNodePrint2((char*)"", pSyncNode); + syncNodeLog2((char*)"", pSyncNode); initRaftId(pSyncNode); diff --git a/source/libs/sync/test/syncRaftCfgTest.cpp b/source/libs/sync/test/syncRaftCfgTest.cpp new file mode 100644 index 0000000000..d3c06fa83e --- /dev/null +++ b/source/libs/sync/test/syncRaftCfgTest.cpp @@ -0,0 +1,101 @@ +#include "syncRaftStore.h" +//#include +#include +#include "syncIO.h" +#include "syncInt.h" +#include "syncRaftCfg.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"); +} + +SSyncCfg* createSyncCfg() { + SSyncCfg* pCfg = (SSyncCfg*)taosMemoryMalloc(sizeof(SSyncCfg)); + memset(pCfg, 0, sizeof(SSyncCfg)); + + pCfg->replicaNum = 3; + pCfg->myIndex = 1; + for (int i = 0; i < pCfg->replicaNum; ++i) { + ((pCfg->nodeInfo)[i]).nodePort = i * 100; + snprintf(((pCfg->nodeInfo)[i]).nodeFqdn, sizeof(((pCfg->nodeInfo)[i]).nodeFqdn), "100.200.300.%d", i); + } + + return pCfg; +} + +void test1() { + SSyncCfg* pCfg = createSyncCfg(); + syncCfgLog2((char*)__FUNCTION__, pCfg); + taosMemoryFree(pCfg); +} + +void test2() { + SSyncCfg* pCfg = createSyncCfg(); + char* s = syncCfg2Str(pCfg); + + SSyncCfg* pCfg2 = (SSyncCfg*)taosMemoryMalloc(sizeof(SSyncCfg)); + int32_t ret = syncCfgFromStr(s, pCfg2); + assert(ret == 0); + syncCfgLog2((char*)__FUNCTION__, pCfg2); + + taosMemoryFree(pCfg); + taosMemoryFree(s); + taosMemoryFree(pCfg2); +} + +void test3() { + SSyncCfg* pCfg = createSyncCfg(); + char* s = (char*)"./test3_raft_cfg.json"; + + if (taosCheckExistFile(s)) { + printf("%s file: %s already exist! \n", (char*)__FUNCTION__, s); + } else { + syncCfgCreateFile(pCfg, s); + printf("%s create json file: %s \n", (char*)__FUNCTION__, s); + } + + taosMemoryFree(pCfg); +} + +void test4() { + SRaftCfg* pCfg = raftCfgOpen("./test3_raft_cfg.json"); + assert(pCfg != NULL); + + raftCfgLog2((char*)__FUNCTION__, pCfg); + + int32_t ret = raftCfgClose(pCfg); + assert(ret == 0); +} + +void test5() { + SRaftCfg* pCfg = raftCfgOpen("./test3_raft_cfg.json"); + assert(pCfg != NULL); + + pCfg->cfg.myIndex = taosGetTimestampSec(); + raftCfgPersist(pCfg); + + printf("%s update json file: %s myIndex->%d \n", (char*)__FUNCTION__, "./test3_raft_cfg.json", pCfg->cfg.myIndex); + + int32_t ret = raftCfgClose(pCfg); + assert(ret == 0); +} + +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/syncRaftStoreTest.cpp b/source/libs/sync/test/syncRaftStoreTest.cpp index 460ff90f4f..070f8c0f07 100644 --- a/source/libs/sync/test/syncRaftStoreTest.cpp +++ b/source/libs/sync/test/syncRaftStoreTest.cpp @@ -30,9 +30,8 @@ void initRaftId() { } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); @@ -40,31 +39,33 @@ int main() { SRaftStore* pRaftStore = raftStoreOpen("./test_raft_store.json"); assert(pRaftStore != NULL); - raftStorePrint2((char*)"==raftStoreOpen==", pRaftStore); + raftStoreLog2((char*)"==raftStoreOpen==", pRaftStore); raftStoreSetTerm(pRaftStore, 100); - raftStorePrint2((char*)"==raftStoreSetTerm==", pRaftStore); + raftStoreLog2((char*)"==raftStoreSetTerm==", pRaftStore); raftStoreVote(pRaftStore, &ids[0]); - raftStorePrint2((char*)"==raftStoreVote==", pRaftStore); + raftStoreLog2((char*)"==raftStoreVote==", pRaftStore); raftStoreClearVote(pRaftStore); - raftStorePrint2((char*)"==raftStoreClearVote==", pRaftStore); + raftStoreLog2((char*)"==raftStoreClearVote==", pRaftStore); raftStoreVote(pRaftStore, &ids[1]); - raftStorePrint2((char*)"==raftStoreVote==", pRaftStore); + raftStoreLog2((char*)"==raftStoreVote==", pRaftStore); raftStoreNextTerm(pRaftStore); - raftStorePrint2((char*)"==raftStoreNextTerm==", pRaftStore); + raftStoreLog2((char*)"==raftStoreNextTerm==", pRaftStore); raftStoreNextTerm(pRaftStore); - raftStorePrint2((char*)"==raftStoreNextTerm==", pRaftStore); + raftStoreLog2((char*)"==raftStoreNextTerm==", pRaftStore); raftStoreNextTerm(pRaftStore); - raftStorePrint2((char*)"==raftStoreNextTerm==", pRaftStore); + raftStoreLog2((char*)"==raftStoreNextTerm==", pRaftStore); raftStoreNextTerm(pRaftStore); - raftStorePrint2((char*)"==raftStoreNextTerm==", pRaftStore); + raftStoreLog2((char*)"==raftStoreNextTerm==", pRaftStore); + + raftStoreClose(pRaftStore); return 0; } diff --git a/source/libs/sync/test/syncReplicateLoadTest.cpp b/source/libs/sync/test/syncReplicateLoadTest.cpp index 0841083b4a..f58c372445 100644 --- a/source/libs/sync/test/syncReplicateLoadTest.cpp +++ b/source/libs/sync/test/syncReplicateLoadTest.cpp @@ -27,29 +27,26 @@ SSyncInfo syncInfo; SSyncFSM *pFsm; SWal * pWal; -void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==CommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), - "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, index, isWeak, - code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, cbMeta.index, + cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==RollBackCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } void initFsm() { @@ -97,8 +94,9 @@ int64_t syncNodeInit() { // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); } - int64_t rid = syncStart(&syncInfo); + int64_t rid = syncOpen(&syncInfo); assert(rid > 0); + syncStart(rid); SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); @@ -141,7 +139,7 @@ SRpcMsg *step0(int i) { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -170,7 +168,7 @@ int main(int argc, char **argv) { SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); - syncNodePrint2((char *)"", pSyncNode); + syncNodeLog2((char *)"", pSyncNode); initRaftId(pSyncNode); // only load ... diff --git a/source/libs/sync/test/syncReplicateTest.cpp b/source/libs/sync/test/syncReplicateTest.cpp index 9783bac7e5..9b5aaf5c18 100644 --- a/source/libs/sync/test/syncReplicateTest.cpp +++ b/source/libs/sync/test/syncReplicateTest.cpp @@ -28,29 +28,26 @@ SSyncFSM * pFsm; SWal * pWal; SSyncNode *gSyncNode; -void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==CommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), - "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, index, isWeak, - code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, cbMeta.index, + cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==RollBackCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } void initFsm() { @@ -136,7 +133,7 @@ SRpcMsg *step0(int i) { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -163,18 +160,18 @@ int main(int argc, char **argv) { gSyncNode = syncInitTest(); assert(gSyncNode != NULL); - syncNodePrint2((char *)"", gSyncNode); + syncNodeLog2((char *)"", gSyncNode); initRaftId(gSyncNode); for (int i = 0; i < 30; ++i) { // step0 SRpcMsg *pMsg0 = step0(i); - syncRpcMsgPrint2((char *)"==step0==", pMsg0); + syncRpcMsgLog2((char *)"==step0==", pMsg0); // step1 SyncClientRequest *pMsg1 = step1(pMsg0); - syncClientRequestPrint2((char *)"==step1==", pMsg1); + syncClientRequestLog2((char *)"==step1==", pMsg1); SyncClientRequest *pSyncClientRequest = pMsg1; SRpcMsg rpcMsg; diff --git a/source/libs/sync/test/syncReplicateTest2.cpp b/source/libs/sync/test/syncReplicateTest2.cpp index 430df7eebd..1e2f823be5 100644 --- a/source/libs/sync/test/syncReplicateTest2.cpp +++ b/source/libs/sync/test/syncReplicateTest2.cpp @@ -27,29 +27,26 @@ SSyncInfo syncInfo; SSyncFSM *pFsm; SWal * pWal; -void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==CommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), - "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, index, isWeak, - code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, cbMeta.index, + cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==RollBackCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } void initFsm() { @@ -97,8 +94,9 @@ int64_t syncNodeInit() { // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); } - int64_t rid = syncStart(&syncInfo); + int64_t rid = syncOpen(&syncInfo); assert(rid > 0); + syncStart(rid); SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); @@ -141,7 +139,7 @@ SRpcMsg *step0(int i) { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -170,13 +168,13 @@ int main(int argc, char **argv) { SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); - syncNodePrint2((char *)"", pSyncNode); + syncNodeLog2((char *)"", pSyncNode); initRaftId(pSyncNode); for (int i = 0; i < 30; ++i) { // step0 SRpcMsg *pMsg0 = step0(i); - syncRpcMsgPrint2((char *)"==step0==", pMsg0); + syncRpcMsgLog2((char *)"==step0==", pMsg0); syncPropose(rid, pMsg0, true); taosMsleep(1000); diff --git a/source/libs/sync/test/syncReplicateTest3.cpp b/source/libs/sync/test/syncReplicateTest3.cpp index 792b9c94cf..e8e24b507d 100644 --- a/source/libs/sync/test/syncReplicateTest3.cpp +++ b/source/libs/sync/test/syncReplicateTest3.cpp @@ -29,29 +29,26 @@ SSyncInfo syncInfo; SSyncFSM *pFsm; SWal * pWal; -void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==CommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), - "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, index, isWeak, - code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, cbMeta.index, + cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==RollBackCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } void initFsm() { @@ -99,8 +96,9 @@ int64_t syncNodeInit() { // taosGetFqdn(pCfg->nodeInfo[0].nodeFqdn); } - int64_t rid = syncStart(&syncInfo); + int64_t rid = syncOpen(&syncInfo); assert(rid > 0); + syncStart(rid); SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); @@ -143,7 +141,7 @@ SRpcMsg *step0(int i) { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -182,13 +180,13 @@ int main(int argc, char **argv) { SSyncNode *pSyncNode = (SSyncNode *)syncNodeAcquire(rid); assert(pSyncNode != NULL); - syncNodePrint2((char *)"", pSyncNode); + syncNodeLog2((char *)"", pSyncNode); initRaftId(pSyncNode); for (int i = 0; i < recordCount; ++i) { // step0 SRpcMsg *pMsg0 = step0(i); - syncRpcMsgPrint2((char *)"==step0==", pMsg0); + syncRpcMsgLog2((char *)"==step0==", pMsg0); syncPropose(rid, pMsg0, true); taosMsleep(sleepMS); diff --git a/source/libs/sync/test/syncRequestVoteReplyTest.cpp b/source/libs/sync/test/syncRequestVoteReplyTest.cpp index 9e0ebee313..d9fe6fa011 100644 --- a/source/libs/sync/test/syncRequestVoteReplyTest.cpp +++ b/source/libs/sync/test/syncRequestVoteReplyTest.cpp @@ -15,7 +15,7 @@ void logTest() { } SyncRequestVoteReply *createMsg() { - SyncRequestVoteReply *pMsg = syncRequestVoteReplyBuild(); + SyncRequestVoteReply *pMsg = syncRequestVoteReplyBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -27,7 +27,7 @@ SyncRequestVoteReply *createMsg() { void test1() { SyncRequestVoteReply *pMsg = createMsg(); - syncRequestVoteReplyPrint2((char *)"test1:", pMsg); + syncRequestVoteReplyLog2((char *)"test1:", pMsg); syncRequestVoteReplyDestroy(pMsg); } @@ -36,9 +36,9 @@ void test2() { uint32_t len = pMsg->bytes; char * serialized = (char *)taosMemoryMalloc(len); syncRequestVoteReplySerialize(pMsg, serialized, len); - SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyBuild(); + SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyBuild(1000); syncRequestVoteReplyDeserialize(serialized, len, pMsg2); - syncRequestVoteReplyPrint2((char *)"test2: syncRequestVoteReplySerialize -> syncRequestVoteReplyDeserialize ", pMsg2); + syncRequestVoteReplyLog2((char *)"test2: syncRequestVoteReplySerialize -> syncRequestVoteReplyDeserialize ", pMsg2); taosMemoryFree(serialized); syncRequestVoteReplyDestroy(pMsg); @@ -50,8 +50,7 @@ void test3() { uint32_t len; char * serialized = syncRequestVoteReplySerialize2(pMsg, &len); SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyDeserialize2(serialized, len); - syncRequestVoteReplyPrint2((char *)"test3: syncRequestVoteReplySerialize3 -> syncRequestVoteReplyDeserialize2 ", - pMsg2); + syncRequestVoteReplyLog2((char *)"test3: syncRequestVoteReplySerialize3 -> syncRequestVoteReplyDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncRequestVoteReplyDestroy(pMsg); @@ -62,10 +61,11 @@ void test4() { SyncRequestVoteReply *pMsg = createMsg(); SRpcMsg rpcMsg; syncRequestVoteReply2RpcMsg(pMsg, &rpcMsg); - SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyBuild(); + SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyBuild(1000); syncRequestVoteReplyFromRpcMsg(&rpcMsg, pMsg2); - syncRequestVoteReplyPrint2((char *)"test4: syncRequestVoteReply2RpcMsg -> syncRequestVoteReplyFromRpcMsg ", pMsg2); + syncRequestVoteReplyLog2((char *)"test4: syncRequestVoteReply2RpcMsg -> syncRequestVoteReplyFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncRequestVoteReplyDestroy(pMsg); syncRequestVoteReplyDestroy(pMsg2); } @@ -75,16 +75,16 @@ void test5() { SRpcMsg rpcMsg; syncRequestVoteReply2RpcMsg(pMsg, &rpcMsg); SyncRequestVoteReply *pMsg2 = syncRequestVoteReplyFromRpcMsg2(&rpcMsg); - syncRequestVoteReplyPrint2((char *)"test5: syncRequestVoteReply2RpcMsg -> syncRequestVoteReplyFromRpcMsg2 ", pMsg2); + syncRequestVoteReplyLog2((char *)"test5: syncRequestVoteReply2RpcMsg -> syncRequestVoteReplyFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncRequestVoteReplyDestroy(pMsg); syncRequestVoteReplyDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncRequestVoteTest.cpp b/source/libs/sync/test/syncRequestVoteTest.cpp index 6b2120e124..4c6d7f94ad 100644 --- a/source/libs/sync/test/syncRequestVoteTest.cpp +++ b/source/libs/sync/test/syncRequestVoteTest.cpp @@ -15,7 +15,7 @@ void logTest() { } SyncRequestVote *createMsg() { - SyncRequestVote *pMsg = syncRequestVoteBuild(); + SyncRequestVote *pMsg = syncRequestVoteBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -28,7 +28,7 @@ SyncRequestVote *createMsg() { void test1() { SyncRequestVote *pMsg = createMsg(); - syncRequestVotePrint2((char *)"test1:", pMsg); + syncRequestVoteLog2((char *)"test1:", pMsg); syncRequestVoteDestroy(pMsg); } @@ -37,9 +37,9 @@ void test2() { uint32_t len = pMsg->bytes; char * serialized = (char *)taosMemoryMalloc(len); syncRequestVoteSerialize(pMsg, serialized, len); - SyncRequestVote *pMsg2 = syncRequestVoteBuild(); + SyncRequestVote *pMsg2 = syncRequestVoteBuild(1000); syncRequestVoteDeserialize(serialized, len, pMsg2); - syncRequestVotePrint2((char *)"test2: syncRequestVoteSerialize -> syncRequestVoteDeserialize ", pMsg2); + syncRequestVoteLog2((char *)"test2: syncRequestVoteSerialize -> syncRequestVoteDeserialize ", pMsg2); taosMemoryFree(serialized); syncRequestVoteDestroy(pMsg); @@ -51,7 +51,7 @@ void test3() { uint32_t len; char * serialized = syncRequestVoteSerialize2(pMsg, &len); SyncRequestVote *pMsg2 = syncRequestVoteDeserialize2(serialized, len); - syncRequestVotePrint2((char *)"test3: syncRequestVoteSerialize3 -> syncRequestVoteDeserialize2 ", pMsg2); + syncRequestVoteLog2((char *)"test3: syncRequestVoteSerialize3 -> syncRequestVoteDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncRequestVoteDestroy(pMsg); @@ -62,10 +62,11 @@ void test4() { SyncRequestVote *pMsg = createMsg(); SRpcMsg rpcMsg; syncRequestVote2RpcMsg(pMsg, &rpcMsg); - SyncRequestVote *pMsg2 = syncRequestVoteBuild(); + SyncRequestVote *pMsg2 = syncRequestVoteBuild(1000); syncRequestVoteFromRpcMsg(&rpcMsg, pMsg2); - syncRequestVotePrint2((char *)"test4: syncRequestVote2RpcMsg -> syncRequestVoteFromRpcMsg ", pMsg2); + syncRequestVoteLog2((char *)"test4: syncRequestVote2RpcMsg -> syncRequestVoteFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncRequestVoteDestroy(pMsg); syncRequestVoteDestroy(pMsg2); } @@ -75,16 +76,16 @@ void test5() { SRpcMsg rpcMsg; syncRequestVote2RpcMsg(pMsg, &rpcMsg); SyncRequestVote *pMsg2 = syncRequestVoteFromRpcMsg2(&rpcMsg); - syncRequestVotePrint2((char *)"test5: syncRequestVote2RpcMsg -> syncRequestVoteFromRpcMsg2 ", pMsg2); + syncRequestVoteLog2((char *)"test5: syncRequestVote2RpcMsg -> syncRequestVoteFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncRequestVoteDestroy(pMsg); syncRequestVoteDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncRespMgrTest.cpp b/source/libs/sync/test/syncRespMgrTest.cpp new file mode 100644 index 0000000000..d22c37d75d --- /dev/null +++ b/source/libs/sync/test/syncRespMgrTest.cpp @@ -0,0 +1,144 @@ +#include "syncRespMgr.h" +//#include +#include +#include "syncIO.h" +#include "syncInt.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"); +} +SSyncRespMgr *pMgr = NULL; + +void syncRespMgrInsert(uint64_t count) { + for (uint64_t i = 0; i < count; ++i) { + SRespStub stub; + memset(&stub, 0, sizeof(SRespStub)); + stub.createTime = taosGetTimestampMs(); + stub.rpcMsg.code = (pMgr->seqNum + 1); + stub.rpcMsg.ahandle = (void *)(200 + i); + stub.rpcMsg.handle = (void *)(300 + i); + uint64_t ret = syncRespMgrAdd(pMgr, &stub); + printf("insert %lu \n", ret); + } +} + +void syncRespMgrDelTest(uint64_t begin, uint64_t end) { + for (uint64_t i = begin; i <= end; ++i) { + int32_t ret = syncRespMgrDel(pMgr, i); + assert(ret == 0); + } +} + +void printStub(SRespStub *p) { + printf("createTime:%ld, rpcMsg.code:%d rpcMsg.ahandle:%ld rpcMsg.handle:%ld \n", p->createTime, p->rpcMsg.code, + (int64_t)(p->rpcMsg.ahandle), (int64_t)(p->rpcMsg.handle)); +} +void syncRespMgrPrint() { + printf("\n----------------syncRespMgrPrint--------------\n"); + taosThreadMutexLock(&(pMgr->mutex)); + + SRespStub *p = (SRespStub *)taosHashIterate(pMgr->pRespHash, NULL); + while (p) { + printStub(p); + p = (SRespStub *)taosHashIterate(pMgr->pRespHash, p); + } + + taosThreadMutexUnlock(&(pMgr->mutex)); +} + +void syncRespMgrGetTest(uint64_t i) { + printf("------syncRespMgrGetTest------- %lu -- \n", i); + SRespStub stub; + int32_t ret = syncRespMgrGet(pMgr, i, &stub); + if (ret == 1) { + printStub(&stub); + } else if (ret == 0) { + printf("%ld notFound \n", i); + } +} + +void syncRespMgrGetAndDelTest(uint64_t i) { + printf("------syncRespMgrGetAndDelTest-------%lu-- \n", i); + SRespStub stub; + int32_t ret = syncRespMgrGetAndDel(pMgr, i, &stub); + if (ret == 1) { + printStub(&stub); + } else if (ret == 0) { + printf("%ld notFound \n", i); + } +} + +void test1() { + printf("------- test1 ---------\n"); + pMgr = syncRespMgrCreate(NULL, 0); + assert(pMgr != NULL); + + syncRespMgrInsert(10); + syncRespMgrPrint(); + + printf("====== get print \n"); + for (uint64_t i = 1; i <= 10; ++i) { + syncRespMgrGetTest(i); + } + + printf("===== delete 5 - 7 \n"); + syncRespMgrDelTest(5, 7); + syncRespMgrPrint(); + + printf("====== get print \n"); + for (uint64_t i = 1; i <= 10; ++i) { + syncRespMgrGetTest(i); + } + + syncRespMgrDestroy(pMgr); +} + +void test2() { + printf("------- test2 ---------\n"); + pMgr = syncRespMgrCreate(NULL, 0); + assert(pMgr != NULL); + + syncRespMgrInsert(10); + syncRespMgrPrint(); + + printf("====== get and delete 3 - 7 \n"); + for (uint64_t i = 3; i <= 7; ++i) { + syncRespMgrGetAndDelTest(i); + } + + syncRespMgrPrint(); + syncRespMgrDestroy(pMgr); +} + +void test3() { + printf("------- test3 ---------\n"); + pMgr = syncRespMgrCreate(NULL, 0); + assert(pMgr != NULL); + + syncRespMgrInsert(10); + syncRespMgrPrint(); + + printf("====== get and delete 0 - 20 \n"); + for (uint64_t i = 0; i <= 20; ++i) { + syncRespMgrGetAndDelTest(i); + } + + syncRespMgrPrint(); + syncRespMgrDestroy(pMgr); +} + +int main() { + tsAsyncLog = 0; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; + logTest(); + test1(); + test2(); + test3(); + + return 0; +} diff --git a/source/libs/sync/test/syncRpcMsgTest.cpp b/source/libs/sync/test/syncRpcMsgTest.cpp index ee6f6c0800..0b21f41080 100644 --- a/source/libs/sync/test/syncRpcMsgTest.cpp +++ b/source/libs/sync/test/syncRpcMsgTest.cpp @@ -16,7 +16,7 @@ void logTest() { int gg = 0; SyncTimeout *createSyncTimeout() { - SyncTimeout *pMsg = syncTimeoutBuild2(SYNC_TIMEOUT_PING, 999, 333, &gg); + SyncTimeout *pMsg = syncTimeoutBuild2(SYNC_TIMEOUT_PING, 999, 333, 1000, &gg); return pMsg; } @@ -26,7 +26,7 @@ SyncPing *createSyncPing() { srcId.vgId = 100; destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); destId.vgId = 100; - SyncPing *pMsg = syncPingBuild3(&srcId, &destId); + SyncPing *pMsg = syncPingBuild3(&srcId, &destId, 1000); return pMsg; } @@ -36,7 +36,7 @@ SyncPingReply *createSyncPingReply() { srcId.vgId = 100; destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); destId.vgId = 100; - SyncPingReply *pMsg = syncPingReplyBuild3(&srcId, &destId); + SyncPingReply *pMsg = syncPingReplyBuild3(&srcId, &destId, 1000); return pMsg; } @@ -47,12 +47,12 @@ SyncClientRequest *createSyncClientRequest() { rpcMsg.contLen = 20; rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); strcpy((char *)rpcMsg.pCont, "hello rpc"); - SyncClientRequest *pMsg = syncClientRequestBuild2(&rpcMsg, 123, true); + SyncClientRequest *pMsg = syncClientRequestBuild2(&rpcMsg, 123, true, 1000); return pMsg; } SyncRequestVote *createSyncRequestVote() { - SyncRequestVote *pMsg = syncRequestVoteBuild(); + SyncRequestVote *pMsg = syncRequestVoteBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -64,7 +64,7 @@ SyncRequestVote *createSyncRequestVote() { } SyncRequestVoteReply *createSyncRequestVoteReply() { - SyncRequestVoteReply *pMsg = syncRequestVoteReplyBuild(); + SyncRequestVoteReply *pMsg = syncRequestVoteReplyBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -75,7 +75,7 @@ SyncRequestVoteReply *createSyncRequestVoteReply() { } SyncAppendEntries *createSyncAppendEntries() { - SyncAppendEntries *pMsg = syncAppendEntriesBuild(20); + SyncAppendEntries *pMsg = syncAppendEntriesBuild(20, 1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -88,7 +88,7 @@ SyncAppendEntries *createSyncAppendEntries() { } SyncAppendEntriesReply *createSyncAppendEntriesReply() { - SyncAppendEntriesReply *pMsg = syncAppendEntriesReplyBuild(); + SyncAppendEntriesReply *pMsg = syncAppendEntriesReplyBuild(1000); pMsg->srcId.addr = syncUtilAddr2U64("127.0.0.1", 1234); pMsg->srcId.vgId = 100; pMsg->destId.addr = syncUtilAddr2U64("127.0.0.1", 5678); @@ -102,7 +102,7 @@ void test1() { SyncTimeout *pMsg = createSyncTimeout(); SRpcMsg rpcMsg; syncTimeout2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test1", &rpcMsg); + syncRpcMsgLog2((char *)"test1", &rpcMsg); syncTimeoutDestroy(pMsg); } @@ -110,7 +110,7 @@ void test2() { SyncPing *pMsg = createSyncPing(); SRpcMsg rpcMsg; syncPing2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test2", &rpcMsg); + syncRpcMsgLog2((char *)"test2", &rpcMsg); syncPingDestroy(pMsg); } @@ -118,7 +118,7 @@ void test3() { SyncPingReply *pMsg = createSyncPingReply(); SRpcMsg rpcMsg; syncPingReply2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test3", &rpcMsg); + syncRpcMsgLog2((char *)"test3", &rpcMsg); syncPingReplyDestroy(pMsg); } @@ -126,7 +126,7 @@ void test4() { SyncRequestVote *pMsg = createSyncRequestVote(); SRpcMsg rpcMsg; syncRequestVote2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test4", &rpcMsg); + syncRpcMsgLog2((char *)"test4", &rpcMsg); syncRequestVoteDestroy(pMsg); } @@ -134,7 +134,7 @@ void test5() { SyncRequestVoteReply *pMsg = createSyncRequestVoteReply(); SRpcMsg rpcMsg; syncRequestVoteReply2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test5", &rpcMsg); + syncRpcMsgLog2((char *)"test5", &rpcMsg); syncRequestVoteReplyDestroy(pMsg); } @@ -142,7 +142,7 @@ void test6() { SyncAppendEntries *pMsg = createSyncAppendEntries(); SRpcMsg rpcMsg; syncAppendEntries2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test6", &rpcMsg); + syncRpcMsgLog2((char *)"test6", &rpcMsg); syncAppendEntriesDestroy(pMsg); } @@ -150,7 +150,7 @@ void test7() { SyncAppendEntriesReply *pMsg = createSyncAppendEntriesReply(); SRpcMsg rpcMsg; syncAppendEntriesReply2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test7", &rpcMsg); + syncRpcMsgLog2((char *)"test7", &rpcMsg); syncAppendEntriesReplyDestroy(pMsg); } @@ -158,14 +158,13 @@ void test8() { SyncClientRequest *pMsg = createSyncClientRequest(); SRpcMsg rpcMsg; syncClientRequest2RpcMsg(pMsg, &rpcMsg); - syncRpcMsgPrint2((char *)"test8", &rpcMsg); + syncRpcMsgLog2((char *)"test8", &rpcMsg); syncClientRequestDestroy(pMsg); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncTest.cpp b/source/libs/sync/test/syncTest.cpp index ec402ff57a..76024e061e 100644 --- a/source/libs/sync/test/syncTest.cpp +++ b/source/libs/sync/test/syncTest.cpp @@ -2,22 +2,62 @@ #include #include "syncIO.h" #include "syncInt.h" -#include "syncRaftStore.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"); +/* +typedef enum { + DEBUG_FATAL = 1, + DEBUG_ERROR = 1, + DEBUG_WARN = 2, + DEBUG_INFO = 2, + DEBUG_DEBUG = 4, + DEBUG_TRACE = 8, + DEBUG_DUMP = 16, + DEBUG_SCREEN = 64, + DEBUG_FILE = 128 +} ELogLevel; +*/ + +void logTest(char* s) { + sFatal("==%s== sync log test: fatal", s); + sError("==%s== sync log test: error", s); + + sWarn("==%s== sync log test: warn", s); + sInfo("==%s== sync log test: info", s); + + sDebug("==%s== sync log test: debug", s); + + sTrace("==%s== sync log test: trace", s); +} + +void test1() { + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; + logTest((char*)__FUNCTION__); +} + +void test2() { + sDebugFlag = DEBUG_DEBUG + DEBUG_SCREEN + DEBUG_FILE; + logTest((char*)__FUNCTION__); +} + +void test3() { + sDebugFlag = DEBUG_INFO + DEBUG_SCREEN + DEBUG_FILE; + logTest((char*)__FUNCTION__); +} + +void test4() { + sDebugFlag = DEBUG_ERROR + DEBUG_SCREEN + DEBUG_FILE; + logTest((char*)__FUNCTION__); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); + // taosInitLog("tmp/syncTest.log", 100); tsAsyncLog = 0; - sDebugFlag = 143 + 64; - logTest(); + test1(); + test2(); + test3(); + test4(); + + // taosCloseLog(); return 0; } diff --git a/source/libs/sync/test/syncTimeoutTest.cpp b/source/libs/sync/test/syncTimeoutTest.cpp index ab36f42d0d..30f25bd1d8 100644 --- a/source/libs/sync/test/syncTimeoutTest.cpp +++ b/source/libs/sync/test/syncTimeoutTest.cpp @@ -17,13 +17,13 @@ void logTest() { int gg = 0; SyncTimeout *createMsg() { - SyncTimeout *pMsg = syncTimeoutBuild2(SYNC_TIMEOUT_PING, 999, 333, &gg); + SyncTimeout *pMsg = syncTimeoutBuild2(SYNC_TIMEOUT_PING, 999, 333, 1000, &gg); return pMsg; } void test1() { SyncTimeout *pMsg = createMsg(); - syncTimeoutPrint2((char *)"test1:", pMsg); + syncTimeoutLog2((char *)"test1:", pMsg); syncTimeoutDestroy(pMsg); } @@ -34,7 +34,7 @@ void test2() { syncTimeoutSerialize(pMsg, serialized, len); SyncTimeout *pMsg2 = syncTimeoutBuild(); syncTimeoutDeserialize(serialized, len, pMsg2); - syncTimeoutPrint2((char *)"test2: syncTimeoutSerialize -> syncTimeoutDeserialize ", pMsg2); + syncTimeoutLog2((char *)"test2: syncTimeoutSerialize -> syncTimeoutDeserialize ", pMsg2); taosMemoryFree(serialized); syncTimeoutDestroy(pMsg); @@ -46,7 +46,7 @@ void test3() { uint32_t len; char * serialized = syncTimeoutSerialize2(pMsg, &len); SyncTimeout *pMsg2 = syncTimeoutDeserialize2(serialized, len); - syncTimeoutPrint2((char *)"test3: syncTimeoutSerialize3 -> syncTimeoutDeserialize2 ", pMsg2); + syncTimeoutLog2((char *)"test3: syncTimeoutSerialize3 -> syncTimeoutDeserialize2 ", pMsg2); taosMemoryFree(serialized); syncTimeoutDestroy(pMsg); @@ -59,8 +59,9 @@ void test4() { syncTimeout2RpcMsg(pMsg, &rpcMsg); SyncTimeout *pMsg2 = (SyncTimeout *)taosMemoryMalloc(rpcMsg.contLen); syncTimeoutFromRpcMsg(&rpcMsg, pMsg2); - syncTimeoutPrint2((char *)"test4: syncTimeout2RpcMsg -> syncTimeoutFromRpcMsg ", pMsg2); + syncTimeoutLog2((char *)"test4: syncTimeout2RpcMsg -> syncTimeoutFromRpcMsg ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncTimeoutDestroy(pMsg); syncTimeoutDestroy(pMsg2); } @@ -70,16 +71,16 @@ void test5() { SRpcMsg rpcMsg; syncTimeout2RpcMsg(pMsg, &rpcMsg); SyncTimeout *pMsg2 = syncTimeoutFromRpcMsg2(&rpcMsg); - syncTimeoutPrint2((char *)"test5: syncTimeout2RpcMsg -> syncTimeoutFromRpcMsg2 ", pMsg2); + syncTimeoutLog2((char *)"test5: syncTimeout2RpcMsg -> syncTimeoutFromRpcMsg2 ", pMsg2); + rpcFreeCont(rpcMsg.pCont); syncTimeoutDestroy(pMsg); syncTimeoutDestroy(pMsg2); } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); test1(); diff --git a/source/libs/sync/test/syncUtilTest.cpp b/source/libs/sync/test/syncUtilTest.cpp index ae2afd2f53..411915fe22 100644 --- a/source/libs/sync/test/syncUtilTest.cpp +++ b/source/libs/sync/test/syncUtilTest.cpp @@ -22,9 +22,8 @@ void electRandomMSTest() { } int main() { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; logTest(); electRandomMSTest(); diff --git a/source/libs/sync/test/syncVotesGrantedTest.cpp b/source/libs/sync/test/syncVotesGrantedTest.cpp index 6164aed351..02a35b3d00 100644 --- a/source/libs/sync/test/syncVotesGrantedTest.cpp +++ b/source/libs/sync/test/syncVotesGrantedTest.cpp @@ -72,9 +72,8 @@ void initRaftId(SSyncNode* pSyncNode) { } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; myIndex = 0; if (argc >= 2) { @@ -118,7 +117,7 @@ int main(int argc, char** argv) { } for (int i = 0; i < replicaNum; ++i) { - SyncRequestVoteReply* reply = syncRequestVoteReplyBuild(); + SyncRequestVoteReply* reply = syncRequestVoteReplyBuild(1000); reply->destId = pSyncNode->myRaftId; reply->srcId = ids[i]; reply->term = term; diff --git a/source/libs/sync/test/syncVotesRespondTest.cpp b/source/libs/sync/test/syncVotesRespondTest.cpp index 39ba6253a1..f276d34745 100644 --- a/source/libs/sync/test/syncVotesRespondTest.cpp +++ b/source/libs/sync/test/syncVotesRespondTest.cpp @@ -72,9 +72,8 @@ void initRaftId(SSyncNode* pSyncNode) { } int main(int argc, char** argv) { - // taosInitLog((char *)"syncTest.log", 100000, 10); tsAsyncLog = 0; - sDebugFlag = 143 + 64; + sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE; myIndex = 0; if (argc >= 2) { @@ -118,7 +117,7 @@ int main(int argc, char** argv) { } for (int i = 0; i < replicaNum; ++i) { - SyncRequestVoteReply* reply = syncRequestVoteReplyBuild(); + SyncRequestVoteReply* reply = syncRequestVoteReplyBuild(1000); reply->destId = pSyncNode->myRaftId; reply->srcId = ids[i]; reply->term = term; diff --git a/source/libs/sync/test/syncWriteTest.cpp b/source/libs/sync/test/syncWriteTest.cpp index a3c24f69ff..cff76a3ae7 100644 --- a/source/libs/sync/test/syncWriteTest.cpp +++ b/source/libs/sync/test/syncWriteTest.cpp @@ -28,29 +28,28 @@ SSyncFSM * pFsm; SWal * pWal; SSyncNode *gSyncNode; -void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +const char *pDir = "./syncWriteTest"; + +void CommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==CommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void PreCommitCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), - "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, index, isWeak, - code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + "==callback== ==PreCommitCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", pFsm, cbMeta.index, + cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } -void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SyncIndex index, bool isWeak, int32_t code, - ESyncState state) { +void RollBackCb(struct SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) { char logBuf[256]; snprintf(logBuf, sizeof(logBuf), "==callback== ==RollBackCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s \n", - pFsm, index, isWeak, code, state, syncUtilState2String(state)); - syncRpcMsgPrint2(logBuf, (SRpcMsg *)pMsg); + pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state)); + syncRpcMsgLog2(logBuf, (SRpcMsg *)pMsg); } void initFsm() { @@ -67,7 +66,7 @@ SSyncNode *syncNodeInit() { syncInfo.queue = gSyncIO->pMsgQ; syncInfo.FpEqMsg = syncIOEqMsg; syncInfo.pFsm = pFsm; - snprintf(syncInfo.path, sizeof(syncInfo.path), "%s", "./write_test"); + snprintf(syncInfo.path, sizeof(syncInfo.path), "%s", pDir); int code = walInit(); assert(code == 0); @@ -108,6 +107,8 @@ SSyncNode *syncNodeInit() { gSyncIO->FpOnSyncTimeout = pSyncNode->FpOnTimeout; gSyncIO->pSyncNode = pSyncNode; + syncNodeStart(pSyncNode); + return pSyncNode; } @@ -133,7 +134,7 @@ SRpcMsg *step0() { } SyncClientRequest *step1(const SRpcMsg *pMsg) { - SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true); + SyncClientRequest *pRetMsg = syncClientRequestBuild2(pMsg, 123, true, 1000); return pRetMsg; } @@ -160,17 +161,17 @@ int main(int argc, char **argv) { gSyncNode = syncInitTest(); assert(gSyncNode != NULL); - syncNodePrint2((char *)"", gSyncNode); + syncNodeLog2((char *)"", gSyncNode); initRaftId(gSyncNode); // step0 SRpcMsg *pMsg0 = step0(); - syncRpcMsgPrint2((char *)"==step0==", pMsg0); + syncRpcMsgLog2((char *)"==step0==", pMsg0); // step1 SyncClientRequest *pMsg1 = step1(pMsg0); - syncClientRequestPrint2((char *)"==step1==", pMsg1); + syncClientRequestLog2((char *)"==step1==", pMsg1); for (int i = 0; i < 10; ++i) { SyncClientRequest *pSyncClientRequest = pMsg1;